mirror of
https://github.com/honeymoose/OpenSearch.git
synced 2025-03-25 17:38:44 +00:00
[ML] Improve response format of data frame stats endpoint (#44743)
This change adjusts the data frame transforms stats endpoint to return a structure that is easier to understand. This is a breaking change for clients of the data frame transforms stats endpoint, but the feature is in beta so stability is not guaranteed. Backport of #44350
This commit is contained in:
parent
6f53865fde
commit
caf9411a72
@ -21,7 +21,7 @@ package org.elasticsearch.client.dataframe;
|
||||
|
||||
import org.elasticsearch.ElasticsearchException;
|
||||
import org.elasticsearch.action.TaskOperationFailure;
|
||||
import org.elasticsearch.client.dataframe.transforms.DataFrameTransformStateAndStats;
|
||||
import org.elasticsearch.client.dataframe.transforms.DataFrameTransformStats;
|
||||
import org.elasticsearch.common.Nullable;
|
||||
import org.elasticsearch.common.ParseField;
|
||||
import org.elasticsearch.common.xcontent.ConstructingObjectParser;
|
||||
@ -42,11 +42,11 @@ public class GetDataFrameTransformStatsResponse {
|
||||
@SuppressWarnings("unchecked")
|
||||
static final ConstructingObjectParser<GetDataFrameTransformStatsResponse, Void> PARSER = new ConstructingObjectParser<>(
|
||||
"get_data_frame_transform_stats_response", true,
|
||||
args -> new GetDataFrameTransformStatsResponse((List<DataFrameTransformStateAndStats>) args[0],
|
||||
args -> new GetDataFrameTransformStatsResponse((List<DataFrameTransformStats>) args[0],
|
||||
(List<TaskOperationFailure>) args[1], (List<ElasticsearchException>) args[2]));
|
||||
|
||||
static {
|
||||
PARSER.declareObjectArray(constructorArg(), DataFrameTransformStateAndStats.PARSER::apply, TRANSFORMS);
|
||||
PARSER.declareObjectArray(constructorArg(), DataFrameTransformStats.PARSER::apply, TRANSFORMS);
|
||||
// Discard the count field which is the size of the transforms array
|
||||
PARSER.declareInt((a, b) -> {}, COUNT);
|
||||
PARSER.declareObjectArray(optionalConstructorArg(), (p, c) -> TaskOperationFailure.fromXContent(p),
|
||||
@ -59,20 +59,20 @@ public class GetDataFrameTransformStatsResponse {
|
||||
return GetDataFrameTransformStatsResponse.PARSER.apply(parser, null);
|
||||
}
|
||||
|
||||
private final List<DataFrameTransformStateAndStats> transformsStateAndStats;
|
||||
private final List<DataFrameTransformStats> transformsStats;
|
||||
private final List<TaskOperationFailure> taskFailures;
|
||||
private final List<ElasticsearchException> nodeFailures;
|
||||
|
||||
public GetDataFrameTransformStatsResponse(List<DataFrameTransformStateAndStats> transformsStateAndStats,
|
||||
public GetDataFrameTransformStatsResponse(List<DataFrameTransformStats> transformsStats,
|
||||
@Nullable List<TaskOperationFailure> taskFailures,
|
||||
@Nullable List<? extends ElasticsearchException> nodeFailures) {
|
||||
this.transformsStateAndStats = transformsStateAndStats;
|
||||
this.transformsStats = transformsStats;
|
||||
this.taskFailures = taskFailures == null ? Collections.emptyList() : Collections.unmodifiableList(taskFailures);
|
||||
this.nodeFailures = nodeFailures == null ? Collections.emptyList() : Collections.unmodifiableList(nodeFailures);
|
||||
}
|
||||
|
||||
public List<DataFrameTransformStateAndStats> getTransformsStateAndStats() {
|
||||
return transformsStateAndStats;
|
||||
public List<DataFrameTransformStats> getTransformsStats() {
|
||||
return transformsStats;
|
||||
}
|
||||
|
||||
public List<ElasticsearchException> getNodeFailures() {
|
||||
@ -85,7 +85,7 @@ public class GetDataFrameTransformStatsResponse {
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Objects.hash(transformsStateAndStats, nodeFailures, taskFailures);
|
||||
return Objects.hash(transformsStats, nodeFailures, taskFailures);
|
||||
}
|
||||
|
||||
@Override
|
||||
@ -99,7 +99,7 @@ public class GetDataFrameTransformStatsResponse {
|
||||
}
|
||||
|
||||
final GetDataFrameTransformStatsResponse that = (GetDataFrameTransformStatsResponse) other;
|
||||
return Objects.equals(this.transformsStateAndStats, that.transformsStateAndStats)
|
||||
return Objects.equals(this.transformsStats, that.transformsStats)
|
||||
&& Objects.equals(this.nodeFailures, that.nodeFailures)
|
||||
&& Objects.equals(this.taskFailures, that.taskFailures);
|
||||
}
|
||||
|
@ -19,48 +19,86 @@
|
||||
|
||||
package org.elasticsearch.client.dataframe.transforms;
|
||||
|
||||
import org.elasticsearch.client.core.IndexerState;
|
||||
import org.elasticsearch.common.ParseField;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.xcontent.ConstructingObjectParser;
|
||||
import org.elasticsearch.common.xcontent.ObjectParser;
|
||||
import org.elasticsearch.common.xcontent.XContentParser;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Objects;
|
||||
|
||||
import static org.elasticsearch.common.xcontent.ConstructingObjectParser.optionalConstructorArg;
|
||||
|
||||
public class DataFrameTransformCheckpointStats {
|
||||
|
||||
public static final ParseField CHECKPOINT = new ParseField("checkpoint");
|
||||
public static final ParseField INDEXER_STATE = new ParseField("indexer_state");
|
||||
public static final ParseField POSITION = new ParseField("position");
|
||||
public static final ParseField CHECKPOINT_PROGRESS = new ParseField("checkpoint_progress");
|
||||
public static final ParseField TIMESTAMP_MILLIS = new ParseField("timestamp_millis");
|
||||
public static final ParseField TIME_UPPER_BOUND_MILLIS = new ParseField("time_upper_bound_millis");
|
||||
|
||||
public static DataFrameTransformCheckpointStats EMPTY = new DataFrameTransformCheckpointStats(0L, 0L);
|
||||
public static final DataFrameTransformCheckpointStats EMPTY = new DataFrameTransformCheckpointStats(0L, null, null, null, 0L, 0L);
|
||||
|
||||
private final long checkpoint;
|
||||
private final IndexerState indexerState;
|
||||
private final DataFrameIndexerPosition position;
|
||||
private final DataFrameTransformProgress checkpointProgress;
|
||||
private final long timestampMillis;
|
||||
private final long timeUpperBoundMillis;
|
||||
|
||||
public static final ConstructingObjectParser<DataFrameTransformCheckpointStats, Void> LENIENT_PARSER = new ConstructingObjectParser<>(
|
||||
"data_frame_transform_checkpoint_stats", true, args -> {
|
||||
long timestamp = args[0] == null ? 0L : (Long) args[0];
|
||||
long timeUpperBound = args[1] == null ? 0L : (Long) args[1];
|
||||
long checkpoint = args[0] == null ? 0L : (Long) args[0];
|
||||
IndexerState indexerState = (IndexerState) args[1];
|
||||
DataFrameIndexerPosition position = (DataFrameIndexerPosition) args[2];
|
||||
DataFrameTransformProgress checkpointProgress = (DataFrameTransformProgress) args[3];
|
||||
long timestamp = args[4] == null ? 0L : (Long) args[4];
|
||||
long timeUpperBound = args[5] == null ? 0L : (Long) args[5];
|
||||
|
||||
return new DataFrameTransformCheckpointStats(timestamp, timeUpperBound);
|
||||
});
|
||||
return new DataFrameTransformCheckpointStats(checkpoint, indexerState, position, checkpointProgress, timestamp, timeUpperBound);
|
||||
});
|
||||
|
||||
static {
|
||||
LENIENT_PARSER.declareLong(ConstructingObjectParser.optionalConstructorArg(), TIMESTAMP_MILLIS);
|
||||
LENIENT_PARSER.declareLong(ConstructingObjectParser.optionalConstructorArg(), TIME_UPPER_BOUND_MILLIS);
|
||||
LENIENT_PARSER.declareLong(optionalConstructorArg(), CHECKPOINT);
|
||||
LENIENT_PARSER.declareField(optionalConstructorArg(), p -> IndexerState.fromString(p.text()), INDEXER_STATE,
|
||||
ObjectParser.ValueType.STRING);
|
||||
LENIENT_PARSER.declareObject(optionalConstructorArg(), DataFrameIndexerPosition.PARSER, POSITION);
|
||||
LENIENT_PARSER.declareObject(optionalConstructorArg(), DataFrameTransformProgress.PARSER, CHECKPOINT_PROGRESS);
|
||||
LENIENT_PARSER.declareLong(optionalConstructorArg(), TIMESTAMP_MILLIS);
|
||||
LENIENT_PARSER.declareLong(optionalConstructorArg(), TIME_UPPER_BOUND_MILLIS);
|
||||
}
|
||||
|
||||
public static DataFrameTransformCheckpointStats fromXContent(XContentParser parser) throws IOException {
|
||||
return LENIENT_PARSER.parse(parser, null);
|
||||
}
|
||||
|
||||
public DataFrameTransformCheckpointStats(final long timestampMillis, final long timeUpperBoundMillis) {
|
||||
public DataFrameTransformCheckpointStats(final long checkpoint, final IndexerState indexerState,
|
||||
final DataFrameIndexerPosition position, final DataFrameTransformProgress checkpointProgress,
|
||||
final long timestampMillis, final long timeUpperBoundMillis) {
|
||||
this.checkpoint = checkpoint;
|
||||
this.indexerState = indexerState;
|
||||
this.position = position;
|
||||
this.checkpointProgress = checkpointProgress;
|
||||
this.timestampMillis = timestampMillis;
|
||||
this.timeUpperBoundMillis = timeUpperBoundMillis;
|
||||
}
|
||||
|
||||
public DataFrameTransformCheckpointStats(StreamInput in) throws IOException {
|
||||
this.timestampMillis = in.readLong();
|
||||
this.timeUpperBoundMillis = in.readLong();
|
||||
public long getCheckpoint() {
|
||||
return checkpoint;
|
||||
}
|
||||
|
||||
public IndexerState getIndexerState() {
|
||||
return indexerState;
|
||||
}
|
||||
|
||||
public DataFrameIndexerPosition getPosition() {
|
||||
return position;
|
||||
}
|
||||
|
||||
public DataFrameTransformProgress getCheckpointProgress() {
|
||||
return checkpointProgress;
|
||||
}
|
||||
|
||||
public long getTimestampMillis() {
|
||||
@ -73,7 +111,7 @@ public class DataFrameTransformCheckpointStats {
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Objects.hash(timestampMillis, timeUpperBoundMillis);
|
||||
return Objects.hash(checkpoint, indexerState, position, checkpointProgress, timestampMillis, timeUpperBoundMillis);
|
||||
}
|
||||
|
||||
@Override
|
||||
@ -88,6 +126,11 @@ public class DataFrameTransformCheckpointStats {
|
||||
|
||||
DataFrameTransformCheckpointStats that = (DataFrameTransformCheckpointStats) other;
|
||||
|
||||
return this.timestampMillis == that.timestampMillis && this.timeUpperBoundMillis == that.timeUpperBoundMillis;
|
||||
return this.checkpoint == that.checkpoint
|
||||
&& Objects.equals(this.indexerState, that.indexerState)
|
||||
&& Objects.equals(this.position, that.position)
|
||||
&& Objects.equals(this.checkpointProgress, that.checkpointProgress)
|
||||
&& this.timestampMillis == that.timestampMillis
|
||||
&& this.timeUpperBoundMillis == that.timeUpperBoundMillis;
|
||||
}
|
||||
}
|
||||
|
@ -27,15 +27,14 @@ import java.util.Objects;
|
||||
|
||||
public class DataFrameTransformCheckpointingInfo {
|
||||
|
||||
public static final ParseField CURRENT_CHECKPOINT = new ParseField("current");
|
||||
public static final ParseField IN_PROGRESS_CHECKPOINT = new ParseField("in_progress");
|
||||
public static final ParseField LAST_CHECKPOINT = new ParseField("last", "current");
|
||||
public static final ParseField NEXT_CHECKPOINT = new ParseField("next", "in_progress");
|
||||
public static final ParseField OPERATIONS_BEHIND = new ParseField("operations_behind");
|
||||
|
||||
private final DataFrameTransformCheckpointStats current;
|
||||
private final DataFrameTransformCheckpointStats inProgress;
|
||||
private final DataFrameTransformCheckpointStats last;
|
||||
private final DataFrameTransformCheckpointStats next;
|
||||
private final long operationsBehind;
|
||||
|
||||
|
||||
private static final ConstructingObjectParser<DataFrameTransformCheckpointingInfo, Void> LENIENT_PARSER =
|
||||
new ConstructingObjectParser<>(
|
||||
"data_frame_transform_checkpointing_info", true, a -> {
|
||||
@ -48,25 +47,25 @@ public class DataFrameTransformCheckpointingInfo {
|
||||
|
||||
static {
|
||||
LENIENT_PARSER.declareObject(ConstructingObjectParser.optionalConstructorArg(),
|
||||
(p, c) -> DataFrameTransformCheckpointStats.fromXContent(p), CURRENT_CHECKPOINT);
|
||||
(p, c) -> DataFrameTransformCheckpointStats.fromXContent(p), LAST_CHECKPOINT);
|
||||
LENIENT_PARSER.declareObject(ConstructingObjectParser.optionalConstructorArg(),
|
||||
(p, c) -> DataFrameTransformCheckpointStats.fromXContent(p), IN_PROGRESS_CHECKPOINT);
|
||||
(p, c) -> DataFrameTransformCheckpointStats.fromXContent(p), NEXT_CHECKPOINT);
|
||||
LENIENT_PARSER.declareLong(ConstructingObjectParser.optionalConstructorArg(), OPERATIONS_BEHIND);
|
||||
}
|
||||
|
||||
public DataFrameTransformCheckpointingInfo(DataFrameTransformCheckpointStats current, DataFrameTransformCheckpointStats inProgress,
|
||||
public DataFrameTransformCheckpointingInfo(DataFrameTransformCheckpointStats last, DataFrameTransformCheckpointStats next,
|
||||
long operationsBehind) {
|
||||
this.current = Objects.requireNonNull(current);
|
||||
this.inProgress = Objects.requireNonNull(inProgress);
|
||||
this.last = Objects.requireNonNull(last);
|
||||
this.next = Objects.requireNonNull(next);
|
||||
this.operationsBehind = operationsBehind;
|
||||
}
|
||||
|
||||
public DataFrameTransformCheckpointStats getCurrent() {
|
||||
return current;
|
||||
public DataFrameTransformCheckpointStats getLast() {
|
||||
return last;
|
||||
}
|
||||
|
||||
public DataFrameTransformCheckpointStats getInProgress() {
|
||||
return inProgress;
|
||||
public DataFrameTransformCheckpointStats getNext() {
|
||||
return next;
|
||||
}
|
||||
|
||||
public long getOperationsBehind() {
|
||||
@ -79,7 +78,7 @@ public class DataFrameTransformCheckpointingInfo {
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Objects.hash(current, inProgress, operationsBehind);
|
||||
return Objects.hash(last, next, operationsBehind);
|
||||
}
|
||||
|
||||
@Override
|
||||
@ -94,8 +93,8 @@ public class DataFrameTransformCheckpointingInfo {
|
||||
|
||||
DataFrameTransformCheckpointingInfo that = (DataFrameTransformCheckpointingInfo) other;
|
||||
|
||||
return Objects.equals(this.current, that.current) &&
|
||||
Objects.equals(this.inProgress, that.inProgress) &&
|
||||
return Objects.equals(this.last, that.last) &&
|
||||
Objects.equals(this.next, that.next) &&
|
||||
this.operationsBehind == that.operationsBehind;
|
||||
}
|
||||
|
||||
|
@ -1,169 +0,0 @@
|
||||
/*
|
||||
* Licensed to Elasticsearch under one or more contributor
|
||||
* license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright
|
||||
* ownership. Elasticsearch licenses this file to you under
|
||||
* the Apache License, Version 2.0 (the "License"); you may
|
||||
* not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.client.dataframe.transforms;
|
||||
|
||||
import org.elasticsearch.client.core.IndexerState;
|
||||
import org.elasticsearch.common.Nullable;
|
||||
import org.elasticsearch.common.ParseField;
|
||||
import org.elasticsearch.common.xcontent.ConstructingObjectParser;
|
||||
import org.elasticsearch.common.xcontent.ObjectParser.ValueType;
|
||||
import org.elasticsearch.common.xcontent.XContentParser;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
|
||||
import static org.elasticsearch.common.xcontent.ConstructingObjectParser.constructorArg;
|
||||
import static org.elasticsearch.common.xcontent.ConstructingObjectParser.optionalConstructorArg;
|
||||
|
||||
public class DataFrameTransformState {
|
||||
|
||||
private static final ParseField INDEXER_STATE = new ParseField("indexer_state");
|
||||
private static final ParseField TASK_STATE = new ParseField("task_state");
|
||||
|
||||
// 7.3 BWC: current_position only exists in 7.2. In 7.3+ it is replaced by position.
|
||||
private static final ParseField CURRENT_POSITION = new ParseField("current_position");
|
||||
private static final ParseField POSITION = new ParseField("position");
|
||||
private static final ParseField CHECKPOINT = new ParseField("checkpoint");
|
||||
private static final ParseField REASON = new ParseField("reason");
|
||||
private static final ParseField PROGRESS = new ParseField("progress");
|
||||
private static final ParseField NODE = new ParseField("node");
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
public static final ConstructingObjectParser<DataFrameTransformState, Void> PARSER =
|
||||
new ConstructingObjectParser<>("data_frame_transform_state", true,
|
||||
args -> {
|
||||
DataFrameTransformTaskState taskState = (DataFrameTransformTaskState) args[0];
|
||||
IndexerState indexerState = (IndexerState) args[1];
|
||||
Map<String, Object> bwcCurrentPosition = (Map<String, Object>) args[2];
|
||||
DataFrameIndexerPosition dataFrameIndexerPosition = (DataFrameIndexerPosition) args[3];
|
||||
|
||||
// BWC handling, translate current_position to position iff position isn't set
|
||||
if (bwcCurrentPosition != null && dataFrameIndexerPosition == null) {
|
||||
dataFrameIndexerPosition = new DataFrameIndexerPosition(bwcCurrentPosition, null);
|
||||
}
|
||||
|
||||
long checkpoint = (long) args[4];
|
||||
String reason = (String) args[5];
|
||||
DataFrameTransformProgress progress = (DataFrameTransformProgress) args[6];
|
||||
NodeAttributes node = (NodeAttributes) args[7];
|
||||
|
||||
return new DataFrameTransformState(taskState, indexerState, dataFrameIndexerPosition, checkpoint, reason, progress,
|
||||
node);
|
||||
});
|
||||
|
||||
static {
|
||||
PARSER.declareField(constructorArg(), p -> DataFrameTransformTaskState.fromString(p.text()), TASK_STATE, ValueType.STRING);
|
||||
PARSER.declareField(constructorArg(), p -> IndexerState.fromString(p.text()), INDEXER_STATE, ValueType.STRING);
|
||||
PARSER.declareField(optionalConstructorArg(), (p, c) -> p.mapOrdered(), CURRENT_POSITION, ValueType.OBJECT);
|
||||
PARSER.declareField(optionalConstructorArg(), DataFrameIndexerPosition::fromXContent, POSITION, ValueType.OBJECT);
|
||||
PARSER.declareLong(ConstructingObjectParser.optionalConstructorArg(), CHECKPOINT);
|
||||
PARSER.declareString(ConstructingObjectParser.optionalConstructorArg(), REASON);
|
||||
PARSER.declareField(optionalConstructorArg(), DataFrameTransformProgress::fromXContent, PROGRESS, ValueType.OBJECT);
|
||||
PARSER.declareField(optionalConstructorArg(), NodeAttributes.PARSER::apply, NODE, ValueType.OBJECT);
|
||||
}
|
||||
|
||||
public static DataFrameTransformState fromXContent(XContentParser parser) throws IOException {
|
||||
return PARSER.parse(parser, null);
|
||||
}
|
||||
|
||||
private final DataFrameTransformTaskState taskState;
|
||||
private final IndexerState indexerState;
|
||||
private final long checkpoint;
|
||||
private final DataFrameIndexerPosition position;
|
||||
private final String reason;
|
||||
private final DataFrameTransformProgress progress;
|
||||
private final NodeAttributes node;
|
||||
|
||||
public DataFrameTransformState(DataFrameTransformTaskState taskState,
|
||||
IndexerState indexerState,
|
||||
@Nullable DataFrameIndexerPosition position,
|
||||
long checkpoint,
|
||||
@Nullable String reason,
|
||||
@Nullable DataFrameTransformProgress progress,
|
||||
@Nullable NodeAttributes node) {
|
||||
this.taskState = taskState;
|
||||
this.indexerState = indexerState;
|
||||
this.position = position;
|
||||
this.checkpoint = checkpoint;
|
||||
this.reason = reason;
|
||||
this.progress = progress;
|
||||
this.node = node;
|
||||
}
|
||||
|
||||
public IndexerState getIndexerState() {
|
||||
return indexerState;
|
||||
}
|
||||
|
||||
public DataFrameTransformTaskState getTaskState() {
|
||||
return taskState;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
public DataFrameIndexerPosition getPosition() {
|
||||
return position;
|
||||
}
|
||||
|
||||
public long getCheckpoint() {
|
||||
return checkpoint;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
public String getReason() {
|
||||
return reason;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
public DataFrameTransformProgress getProgress() {
|
||||
return progress;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
public NodeAttributes getNode() {
|
||||
return node;
|
||||
}
|
||||
|
||||
@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.taskState, that.taskState) &&
|
||||
Objects.equals(this.indexerState, that.indexerState) &&
|
||||
Objects.equals(this.position, that.position) &&
|
||||
Objects.equals(this.progress, that.progress) &&
|
||||
this.checkpoint == that.checkpoint &&
|
||||
Objects.equals(this.node, that.node) &&
|
||||
Objects.equals(this.reason, that.reason);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Objects.hash(taskState, indexerState, position, checkpoint, reason, progress, node);
|
||||
}
|
||||
|
||||
}
|
@ -1,104 +0,0 @@
|
||||
/*
|
||||
* Licensed to Elasticsearch under one or more contributor
|
||||
* license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright
|
||||
* ownership. Elasticsearch licenses this file to you under
|
||||
* the Apache License, Version 2.0 (the "License"); you may
|
||||
* not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.client.dataframe.transforms;
|
||||
|
||||
import org.elasticsearch.common.ParseField;
|
||||
import org.elasticsearch.common.xcontent.ConstructingObjectParser;
|
||||
import org.elasticsearch.common.xcontent.XContentParser;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Objects;
|
||||
|
||||
public class DataFrameTransformStateAndStats {
|
||||
|
||||
public static final ParseField ID = new ParseField("id");
|
||||
public static final ParseField STATE_FIELD = new ParseField("state");
|
||||
public static final ParseField STATS_FIELD = new ParseField("stats");
|
||||
public static final ParseField CHECKPOINTING_INFO_FIELD = new ParseField("checkpointing");
|
||||
|
||||
public static final ConstructingObjectParser<DataFrameTransformStateAndStats, Void> PARSER = new ConstructingObjectParser<>(
|
||||
"data_frame_transform_state_and_stats", true,
|
||||
a -> new DataFrameTransformStateAndStats((String) a[0], (DataFrameTransformState) a[1], (DataFrameIndexerTransformStats) a[2],
|
||||
(DataFrameTransformCheckpointingInfo) a[3]));
|
||||
|
||||
static {
|
||||
PARSER.declareString(ConstructingObjectParser.constructorArg(), ID);
|
||||
PARSER.declareObject(ConstructingObjectParser.constructorArg(), DataFrameTransformState.PARSER::apply, STATE_FIELD);
|
||||
PARSER.declareObject(ConstructingObjectParser.constructorArg(), (p, c) -> DataFrameIndexerTransformStats.fromXContent(p),
|
||||
STATS_FIELD);
|
||||
PARSER.declareObject(ConstructingObjectParser.optionalConstructorArg(),
|
||||
(p, c) -> DataFrameTransformCheckpointingInfo.fromXContent(p), CHECKPOINTING_INFO_FIELD);
|
||||
}
|
||||
|
||||
public static DataFrameTransformStateAndStats fromXContent(XContentParser parser) throws IOException {
|
||||
return PARSER.parse(parser, null);
|
||||
}
|
||||
|
||||
private final String id;
|
||||
private final DataFrameTransformState transformState;
|
||||
private final DataFrameIndexerTransformStats transformStats;
|
||||
private final DataFrameTransformCheckpointingInfo checkpointingInfo;
|
||||
|
||||
public DataFrameTransformStateAndStats(String id, DataFrameTransformState state, DataFrameIndexerTransformStats stats,
|
||||
DataFrameTransformCheckpointingInfo checkpointingInfo) {
|
||||
this.id = id;
|
||||
this.transformState = state;
|
||||
this.transformStats = stats;
|
||||
this.checkpointingInfo = checkpointingInfo;
|
||||
}
|
||||
|
||||
public String getId() {
|
||||
return id;
|
||||
}
|
||||
|
||||
public DataFrameIndexerTransformStats getTransformStats() {
|
||||
return transformStats;
|
||||
}
|
||||
|
||||
public DataFrameTransformState getTransformState() {
|
||||
return transformState;
|
||||
}
|
||||
|
||||
public DataFrameTransformCheckpointingInfo getCheckpointingInfo() {
|
||||
return checkpointingInfo;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Objects.hash(id, transformState, transformStats, checkpointingInfo);
|
||||
}
|
||||
|
||||
@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)
|
||||
&& Objects.equals(this.checkpointingInfo, that.checkpointingInfo);
|
||||
}
|
||||
}
|
@ -0,0 +1,128 @@
|
||||
/*
|
||||
* Licensed to Elasticsearch under one or more contributor
|
||||
* license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright
|
||||
* ownership. Elasticsearch licenses this file to you under
|
||||
* the Apache License, Version 2.0 (the "License"); you may
|
||||
* not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.client.dataframe.transforms;
|
||||
|
||||
import org.elasticsearch.common.ParseField;
|
||||
import org.elasticsearch.common.xcontent.ConstructingObjectParser;
|
||||
import org.elasticsearch.common.xcontent.ObjectParser;
|
||||
import org.elasticsearch.common.xcontent.XContentParser;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Objects;
|
||||
|
||||
import static org.elasticsearch.common.xcontent.ConstructingObjectParser.constructorArg;
|
||||
import static org.elasticsearch.common.xcontent.ConstructingObjectParser.optionalConstructorArg;
|
||||
|
||||
public class DataFrameTransformStats {
|
||||
|
||||
public static final ParseField ID = new ParseField("id");
|
||||
public static final ParseField TASK_STATE_FIELD = new ParseField("task_state");
|
||||
public static final ParseField REASON_FIELD = new ParseField("reason");
|
||||
public static final ParseField NODE_FIELD = new ParseField("node");
|
||||
public static final ParseField STATS_FIELD = new ParseField("stats");
|
||||
public static final ParseField CHECKPOINTING_INFO_FIELD = new ParseField("checkpointing");
|
||||
|
||||
public static final ConstructingObjectParser<DataFrameTransformStats, Void> PARSER = new ConstructingObjectParser<>(
|
||||
"data_frame_transform_state_and_stats_info", true,
|
||||
a -> new DataFrameTransformStats((String) a[0], (DataFrameTransformTaskState) a[1], (String) a[2],
|
||||
(NodeAttributes) a[3], (DataFrameIndexerTransformStats) a[4], (DataFrameTransformCheckpointingInfo) a[5]));
|
||||
|
||||
static {
|
||||
PARSER.declareString(constructorArg(), ID);
|
||||
PARSER.declareField(optionalConstructorArg(), p -> DataFrameTransformTaskState.fromString(p.text()), TASK_STATE_FIELD,
|
||||
ObjectParser.ValueType.STRING);
|
||||
PARSER.declareString(optionalConstructorArg(), REASON_FIELD);
|
||||
PARSER.declareField(optionalConstructorArg(), NodeAttributes.PARSER::apply, NODE_FIELD, ObjectParser.ValueType.OBJECT);
|
||||
PARSER.declareObject(constructorArg(), (p, c) -> DataFrameIndexerTransformStats.fromXContent(p), STATS_FIELD);
|
||||
PARSER.declareObject(optionalConstructorArg(),
|
||||
(p, c) -> DataFrameTransformCheckpointingInfo.fromXContent(p), CHECKPOINTING_INFO_FIELD);
|
||||
}
|
||||
|
||||
public static DataFrameTransformStats fromXContent(XContentParser parser) throws IOException {
|
||||
return PARSER.parse(parser, null);
|
||||
}
|
||||
|
||||
private final String id;
|
||||
private final String reason;
|
||||
private final DataFrameTransformTaskState taskState;
|
||||
private final NodeAttributes node;
|
||||
private final DataFrameIndexerTransformStats indexerStats;
|
||||
private final DataFrameTransformCheckpointingInfo checkpointingInfo;
|
||||
|
||||
public DataFrameTransformStats(String id, DataFrameTransformTaskState taskState, String reason, NodeAttributes node,
|
||||
DataFrameIndexerTransformStats stats,
|
||||
DataFrameTransformCheckpointingInfo checkpointingInfo) {
|
||||
this.id = id;
|
||||
this.taskState = taskState;
|
||||
this.reason = reason;
|
||||
this.node = node;
|
||||
this.indexerStats = stats;
|
||||
this.checkpointingInfo = checkpointingInfo;
|
||||
}
|
||||
|
||||
public String getId() {
|
||||
return id;
|
||||
}
|
||||
|
||||
public DataFrameTransformTaskState getTaskState() {
|
||||
return taskState;
|
||||
}
|
||||
|
||||
public String getReason() {
|
||||
return reason;
|
||||
}
|
||||
|
||||
public NodeAttributes getNode() {
|
||||
return node;
|
||||
}
|
||||
|
||||
public DataFrameIndexerTransformStats getIndexerStats() {
|
||||
return indexerStats;
|
||||
}
|
||||
|
||||
public DataFrameTransformCheckpointingInfo getCheckpointingInfo() {
|
||||
return checkpointingInfo;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Objects.hash(id, taskState, reason, node, indexerStats, checkpointingInfo);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object other) {
|
||||
if (this == other) {
|
||||
return true;
|
||||
}
|
||||
|
||||
if (other == null || getClass() != other.getClass()) {
|
||||
return false;
|
||||
}
|
||||
|
||||
DataFrameTransformStats that = (DataFrameTransformStats) other;
|
||||
|
||||
return Objects.equals(this.id, that.id)
|
||||
&& Objects.equals(this.taskState, that.taskState)
|
||||
&& Objects.equals(this.reason, that.reason)
|
||||
&& Objects.equals(this.node, that.node)
|
||||
&& Objects.equals(this.indexerStats, that.indexerStats)
|
||||
&& Objects.equals(this.checkpointingInfo, that.checkpointingInfo);
|
||||
}
|
||||
}
|
@ -25,7 +25,6 @@ import org.elasticsearch.action.bulk.BulkResponse;
|
||||
import org.elasticsearch.action.index.IndexRequest;
|
||||
import org.elasticsearch.action.support.WriteRequest;
|
||||
import org.elasticsearch.client.core.AcknowledgedResponse;
|
||||
import org.elasticsearch.client.core.IndexerState;
|
||||
import org.elasticsearch.client.core.PageParams;
|
||||
import org.elasticsearch.client.dataframe.DeleteDataFrameTransformRequest;
|
||||
import org.elasticsearch.client.dataframe.GetDataFrameTransformRequest;
|
||||
@ -41,7 +40,7 @@ import org.elasticsearch.client.dataframe.StopDataFrameTransformRequest;
|
||||
import org.elasticsearch.client.dataframe.StopDataFrameTransformResponse;
|
||||
import org.elasticsearch.client.dataframe.transforms.DataFrameIndexerTransformStats;
|
||||
import org.elasticsearch.client.dataframe.transforms.DataFrameTransformConfig;
|
||||
import org.elasticsearch.client.dataframe.transforms.DataFrameTransformStateAndStats;
|
||||
import org.elasticsearch.client.dataframe.transforms.DataFrameTransformStats;
|
||||
import org.elasticsearch.client.dataframe.transforms.DataFrameTransformTaskState;
|
||||
import org.elasticsearch.client.dataframe.transforms.DestConfig;
|
||||
import org.elasticsearch.client.dataframe.transforms.SourceConfig;
|
||||
@ -275,8 +274,8 @@ public class DataFrameTransformIT extends ESRestHighLevelClientTestCase {
|
||||
|
||||
GetDataFrameTransformStatsResponse statsResponse = execute(new GetDataFrameTransformStatsRequest(id),
|
||||
client::getDataFrameTransformStats, client::getDataFrameTransformStatsAsync);
|
||||
assertThat(statsResponse.getTransformsStateAndStats(), hasSize(1));
|
||||
DataFrameTransformTaskState taskState = statsResponse.getTransformsStateAndStats().get(0).getTransformState().getTaskState();
|
||||
assertThat(statsResponse.getTransformsStats(), hasSize(1));
|
||||
DataFrameTransformTaskState taskState = statsResponse.getTransformsStats().get(0).getTaskState();
|
||||
|
||||
// Since we are non-continuous, the transform could auto-stop between being started earlier and us gathering the statistics
|
||||
assertThat(taskState, is(oneOf(DataFrameTransformTaskState.STARTED, DataFrameTransformTaskState.STOPPED)));
|
||||
@ -291,7 +290,7 @@ public class DataFrameTransformIT extends ESRestHighLevelClientTestCase {
|
||||
// Calling stop with wait_for_completion assures that we will be in the `STOPPED` state for the transform task
|
||||
statsResponse = execute(new GetDataFrameTransformStatsRequest(id),
|
||||
client::getDataFrameTransformStats, client::getDataFrameTransformStatsAsync);
|
||||
taskState = statsResponse.getTransformsStateAndStats().get(0).getTransformState().getTaskState();
|
||||
taskState = statsResponse.getTransformsStats().get(0).getTaskState();
|
||||
assertThat(taskState, is(DataFrameTransformTaskState.STOPPED));
|
||||
}
|
||||
|
||||
@ -370,13 +369,12 @@ public class DataFrameTransformIT extends ESRestHighLevelClientTestCase {
|
||||
GetDataFrameTransformStatsResponse statsResponse = execute(new GetDataFrameTransformStatsRequest(id),
|
||||
client::getDataFrameTransformStats, client::getDataFrameTransformStatsAsync);
|
||||
|
||||
assertEquals(1, statsResponse.getTransformsStateAndStats().size());
|
||||
DataFrameTransformStateAndStats stats = statsResponse.getTransformsStateAndStats().get(0);
|
||||
assertEquals(DataFrameTransformTaskState.STOPPED, stats.getTransformState().getTaskState());
|
||||
assertEquals(IndexerState.STOPPED, stats.getTransformState().getIndexerState());
|
||||
assertEquals(1, statsResponse.getTransformsStats().size());
|
||||
DataFrameTransformStats stats = statsResponse.getTransformsStats().get(0);
|
||||
assertEquals(DataFrameTransformTaskState.STOPPED, stats.getTaskState());
|
||||
|
||||
DataFrameIndexerTransformStats zeroIndexerStats = new DataFrameIndexerTransformStats(0L, 0L, 0L, 0L, 0L, 0L, 0L, 0L, 0L, 0L);
|
||||
assertEquals(zeroIndexerStats, stats.getTransformStats());
|
||||
assertEquals(zeroIndexerStats, stats.getIndexerStats());
|
||||
|
||||
// start the transform
|
||||
StartDataFrameTransformResponse startTransformResponse = execute(new StartDataFrameTransformRequest(id),
|
||||
@ -386,17 +384,15 @@ public class DataFrameTransformIT extends ESRestHighLevelClientTestCase {
|
||||
assertBusy(() -> {
|
||||
GetDataFrameTransformStatsResponse response = execute(new GetDataFrameTransformStatsRequest(id),
|
||||
client::getDataFrameTransformStats, client::getDataFrameTransformStatsAsync);
|
||||
DataFrameTransformStateAndStats stateAndStats = response.getTransformsStateAndStats().get(0);
|
||||
assertNotEquals(zeroIndexerStats, stateAndStats.getTransformStats());
|
||||
assertNotNull(stateAndStats.getTransformState().getProgress());
|
||||
assertThat(stateAndStats.getTransformState().getTaskState(),
|
||||
DataFrameTransformStats stateAndStats = response.getTransformsStats().get(0);
|
||||
assertNotEquals(zeroIndexerStats, stateAndStats.getIndexerStats());
|
||||
assertThat(stateAndStats.getTaskState(),
|
||||
is(oneOf(DataFrameTransformTaskState.STARTED, DataFrameTransformTaskState.STOPPED)));
|
||||
assertThat(stateAndStats.getTransformState().getIndexerState(),
|
||||
is(oneOf(IndexerState.STARTED, IndexerState.STOPPED)));
|
||||
assertThat(stateAndStats.getTransformState().getProgress().getPercentComplete(), equalTo(100.0));
|
||||
assertThat(stateAndStats.getTransformState().getProgress().getTotalDocs(), greaterThan(0L));
|
||||
assertThat(stateAndStats.getTransformState().getProgress().getRemainingDocs(), equalTo(0L));
|
||||
assertThat(stateAndStats.getTransformState().getReason(), is(nullValue()));
|
||||
assertNotNull(stateAndStats.getCheckpointingInfo().getNext().getCheckpointProgress());
|
||||
assertThat(stateAndStats.getCheckpointingInfo().getNext().getCheckpointProgress().getPercentComplete(), equalTo(100.0));
|
||||
assertThat(stateAndStats.getCheckpointingInfo().getNext().getCheckpointProgress().getTotalDocs(), greaterThan(0L));
|
||||
assertThat(stateAndStats.getCheckpointingInfo().getNext().getCheckpointProgress().getRemainingDocs(), equalTo(0L));
|
||||
assertThat(stateAndStats.getReason(), is(nullValue()));
|
||||
});
|
||||
}
|
||||
|
||||
|
@ -21,8 +21,8 @@ package org.elasticsearch.client.dataframe;
|
||||
|
||||
import org.elasticsearch.ElasticsearchException;
|
||||
import org.elasticsearch.action.TaskOperationFailure;
|
||||
import org.elasticsearch.client.dataframe.transforms.DataFrameTransformStateAndStats;
|
||||
import org.elasticsearch.client.dataframe.transforms.DataFrameTransformStateAndStatsTests;
|
||||
import org.elasticsearch.client.dataframe.transforms.DataFrameTransformStats;
|
||||
import org.elasticsearch.client.dataframe.transforms.DataFrameTransformStatsTests;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
|
||||
@ -48,9 +48,9 @@ public class GetDataFrameTransformStatsResponseTests extends ESTestCase {
|
||||
|
||||
private static GetDataFrameTransformStatsResponse createTestInstance() {
|
||||
int count = randomIntBetween(1, 3);
|
||||
List<DataFrameTransformStateAndStats> stats = new ArrayList<>();
|
||||
List<DataFrameTransformStats> stats = new ArrayList<>();
|
||||
for (int i=0; i<count; i++) {
|
||||
stats.add(DataFrameTransformStateAndStatsTests.randomInstance());
|
||||
stats.add(DataFrameTransformStatsTests.randomInstance());
|
||||
}
|
||||
|
||||
List<TaskOperationFailure> taskFailures = null;
|
||||
@ -77,8 +77,8 @@ public class GetDataFrameTransformStatsResponseTests extends ESTestCase {
|
||||
builder.startObject();
|
||||
{
|
||||
builder.startArray("transforms");
|
||||
for (DataFrameTransformStateAndStats stats : response.getTransformsStateAndStats()) {
|
||||
DataFrameTransformStateAndStatsTests.toXContent(stats, builder);
|
||||
for (DataFrameTransformStats stats : response.getTransformsStats()) {
|
||||
DataFrameTransformStatsTests.toXContent(stats, builder);
|
||||
}
|
||||
builder.endArray();
|
||||
|
||||
@ -92,7 +92,7 @@ public class GetDataFrameTransformStatsResponseTests extends ESTestCase {
|
||||
// the object so use a custom compare method rather than Object.equals
|
||||
private static void assertEqualInstances(GetDataFrameTransformStatsResponse expected,
|
||||
GetDataFrameTransformStatsResponse actual) {
|
||||
assertEquals(expected.getTransformsStateAndStats(), actual.getTransformsStateAndStats());
|
||||
assertEquals(expected.getTransformsStats(), actual.getTransformsStats());
|
||||
AcknowledgedTasksResponseTests.assertTaskOperationFailuresEqual(expected.getTaskFailures(), actual.getTaskFailures());
|
||||
AcknowledgedTasksResponseTests.assertNodeFailuresEqual(expected.getNodeFailures(), actual.getNodeFailures());
|
||||
}
|
||||
|
@ -19,6 +19,7 @@
|
||||
|
||||
package org.elasticsearch.client.dataframe.transforms;
|
||||
|
||||
import org.elasticsearch.client.core.IndexerState;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
|
||||
@ -30,22 +31,38 @@ public class DataFrameTransformCheckpointStatsTests extends ESTestCase {
|
||||
|
||||
public void testFromXContent() throws IOException {
|
||||
xContentTester(this::createParser,
|
||||
DataFrameTransformCheckpointStatsTests::randomDataFrameTransformCheckpointStats,
|
||||
DataFrameTransformCheckpointStatsTests::toXContent,
|
||||
DataFrameTransformCheckpointStats::fromXContent)
|
||||
DataFrameTransformCheckpointStatsTests::randomDataFrameTransformCheckpointStats,
|
||||
DataFrameTransformCheckpointStatsTests::toXContent,
|
||||
DataFrameTransformCheckpointStats::fromXContent)
|
||||
.supportsUnknownFields(true)
|
||||
.randomFieldsExcludeFilter(field -> field.startsWith("position"))
|
||||
.test();
|
||||
}
|
||||
|
||||
public static DataFrameTransformCheckpointStats randomDataFrameTransformCheckpointStats() {
|
||||
return new DataFrameTransformCheckpointStats(randomLongBetween(1, 1_000_000), randomLongBetween(0, 1_000_000));
|
||||
return new DataFrameTransformCheckpointStats(randomLongBetween(1, 1_000_000),
|
||||
randomBoolean() ? null : randomFrom(IndexerState.values()),
|
||||
randomBoolean() ? null : DataFrameIndexerPositionTests.randomDataFrameIndexerPosition(),
|
||||
randomBoolean() ? null : DataFrameTransformProgressTests.randomInstance(),
|
||||
randomLongBetween(1, 1_000_000), randomLongBetween(0, 1_000_000));
|
||||
}
|
||||
|
||||
public static void toXContent(DataFrameTransformCheckpointStats stats, XContentBuilder builder) throws IOException {
|
||||
builder.startObject();
|
||||
builder.field("timestamp_millis", stats.getTimestampMillis());
|
||||
builder.field("time_upper_bound_millis", stats.getTimeUpperBoundMillis());
|
||||
builder.field(DataFrameTransformCheckpointStats.CHECKPOINT.getPreferredName(), stats.getCheckpoint());
|
||||
if (stats.getIndexerState() != null) {
|
||||
builder.field(DataFrameTransformCheckpointStats.INDEXER_STATE.getPreferredName(), stats.getIndexerState().value());
|
||||
}
|
||||
if (stats.getPosition() != null) {
|
||||
builder.field(DataFrameTransformCheckpointStats.POSITION.getPreferredName());
|
||||
DataFrameIndexerPositionTests.toXContent(stats.getPosition(), builder);
|
||||
}
|
||||
if (stats.getCheckpointProgress() != null) {
|
||||
builder.field(DataFrameTransformCheckpointStats.CHECKPOINT_PROGRESS.getPreferredName());
|
||||
DataFrameTransformProgressTests.toXContent(stats.getCheckpointProgress(), builder);
|
||||
}
|
||||
builder.field(DataFrameTransformCheckpointStats.TIMESTAMP_MILLIS.getPreferredName(), stats.getTimestampMillis());
|
||||
builder.field(DataFrameTransformCheckpointStats.TIME_UPPER_BOUND_MILLIS.getPreferredName(), stats.getTimeUpperBoundMillis());
|
||||
builder.endObject();
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
@ -30,32 +30,31 @@ public class DataFrameTransformCheckpointingInfoTests extends ESTestCase {
|
||||
|
||||
public void testFromXContent() throws IOException {
|
||||
xContentTester(this::createParser,
|
||||
DataFrameTransformCheckpointingInfoTests::randomDataFrameTransformCheckpointingInfo,
|
||||
DataFrameTransformCheckpointingInfoTests::toXContent,
|
||||
DataFrameTransformCheckpointingInfo::fromXContent)
|
||||
DataFrameTransformCheckpointingInfoTests::randomDataFrameTransformCheckpointingInfo,
|
||||
DataFrameTransformCheckpointingInfoTests::toXContent,
|
||||
DataFrameTransformCheckpointingInfo::fromXContent)
|
||||
.supportsUnknownFields(false)
|
||||
.test();
|
||||
}
|
||||
|
||||
public static DataFrameTransformCheckpointingInfo randomDataFrameTransformCheckpointingInfo() {
|
||||
return new DataFrameTransformCheckpointingInfo(
|
||||
DataFrameTransformCheckpointStatsTests.randomDataFrameTransformCheckpointStats(),
|
||||
DataFrameTransformCheckpointStatsTests.randomDataFrameTransformCheckpointStats(),
|
||||
randomLongBetween(0, 10000));
|
||||
DataFrameTransformCheckpointStatsTests.randomDataFrameTransformCheckpointStats(),
|
||||
DataFrameTransformCheckpointStatsTests.randomDataFrameTransformCheckpointStats(),
|
||||
randomLongBetween(0, 10000));
|
||||
}
|
||||
|
||||
public static void toXContent(DataFrameTransformCheckpointingInfo info, XContentBuilder builder) throws IOException {
|
||||
builder.startObject();
|
||||
if (info.getCurrent().getTimestampMillis() > 0) {
|
||||
builder.field("current");
|
||||
DataFrameTransformCheckpointStatsTests.toXContent(info.getCurrent(), builder);
|
||||
if (info.getLast().getTimestampMillis() > 0) {
|
||||
builder.field(DataFrameTransformCheckpointingInfo.LAST_CHECKPOINT.getPreferredName());
|
||||
DataFrameTransformCheckpointStatsTests.toXContent(info.getLast(), builder);
|
||||
}
|
||||
if (info.getInProgress().getTimestampMillis() > 0) {
|
||||
builder.field("in_progress");
|
||||
DataFrameTransformCheckpointStatsTests.toXContent(info.getInProgress(), builder);
|
||||
if (info.getNext().getTimestampMillis() > 0) {
|
||||
builder.field(DataFrameTransformCheckpointingInfo.NEXT_CHECKPOINT.getPreferredName());
|
||||
DataFrameTransformCheckpointStatsTests.toXContent(info.getNext(), builder);
|
||||
}
|
||||
builder.field("operations_behind", info.getOperationsBehind());
|
||||
builder.field(DataFrameTransformCheckpointingInfo.OPERATIONS_BEHIND.getPreferredName(), info.getOperationsBehind());
|
||||
builder.endObject();
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -1,59 +0,0 @@
|
||||
/*
|
||||
* Licensed to Elasticsearch under one or more contributor
|
||||
* license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright
|
||||
* ownership. Elasticsearch licenses this file to you under
|
||||
* the Apache License, Version 2.0 (the "License"); you may
|
||||
* not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.client.dataframe.transforms;
|
||||
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import static org.elasticsearch.test.AbstractXContentTestCase.xContentTester;
|
||||
|
||||
public class DataFrameTransformStateAndStatsTests extends ESTestCase {
|
||||
|
||||
public void testFromXContent() throws IOException {
|
||||
xContentTester(this::createParser,
|
||||
DataFrameTransformStateAndStatsTests::randomInstance,
|
||||
DataFrameTransformStateAndStatsTests::toXContent,
|
||||
DataFrameTransformStateAndStats::fromXContent)
|
||||
.supportsUnknownFields(true)
|
||||
.randomFieldsExcludeFilter(field -> field.startsWith("state"))
|
||||
.test();
|
||||
}
|
||||
|
||||
public static DataFrameTransformStateAndStats randomInstance() {
|
||||
return new DataFrameTransformStateAndStats(randomAlphaOfLength(10),
|
||||
DataFrameTransformStateTests.randomDataFrameTransformState(),
|
||||
DataFrameIndexerTransformStatsTests.randomStats(),
|
||||
DataFrameTransformCheckpointingInfoTests.randomDataFrameTransformCheckpointingInfo());
|
||||
}
|
||||
|
||||
public static void toXContent(DataFrameTransformStateAndStats stateAndStats, XContentBuilder builder) throws IOException {
|
||||
builder.startObject();
|
||||
builder.field(DataFrameTransformStateAndStats.ID.getPreferredName(), stateAndStats.getId());
|
||||
builder.field(DataFrameTransformStateAndStats.STATE_FIELD.getPreferredName());
|
||||
DataFrameTransformStateTests.toXContent(stateAndStats.getTransformState(), builder);
|
||||
builder.field(DataFrameTransformStateAndStats.STATS_FIELD.getPreferredName());
|
||||
DataFrameIndexerTransformStatsTests.toXContent(stateAndStats.getTransformStats(), builder);
|
||||
builder.field(DataFrameTransformStateAndStats.CHECKPOINTING_INFO_FIELD.getPreferredName());
|
||||
DataFrameTransformCheckpointingInfoTests.toXContent(stateAndStats.getCheckpointingInfo(), builder);
|
||||
builder.endObject();
|
||||
}
|
||||
}
|
@ -1,78 +0,0 @@
|
||||
/*
|
||||
* Licensed to Elasticsearch under one or more contributor
|
||||
* license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright
|
||||
* ownership. Elasticsearch licenses this file to you under
|
||||
* the Apache License, Version 2.0 (the "License"); you may
|
||||
* not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.client.dataframe.transforms;
|
||||
|
||||
import org.elasticsearch.client.core.IndexerState;
|
||||
import org.elasticsearch.common.xcontent.ToXContent;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import static org.elasticsearch.test.AbstractXContentTestCase.xContentTester;
|
||||
|
||||
public class DataFrameTransformStateTests extends ESTestCase {
|
||||
|
||||
public void testFromXContent() throws IOException {
|
||||
xContentTester(this::createParser,
|
||||
DataFrameTransformStateTests::randomDataFrameTransformState,
|
||||
DataFrameTransformStateTests::toXContent,
|
||||
DataFrameTransformState::fromXContent)
|
||||
.supportsUnknownFields(true)
|
||||
.randomFieldsExcludeFilter(field -> field.equals("position.indexer_position") ||
|
||||
field.equals("position.bucket_position") ||
|
||||
field.equals("node.attributes"))
|
||||
.test();
|
||||
}
|
||||
|
||||
public static DataFrameTransformState randomDataFrameTransformState() {
|
||||
return new DataFrameTransformState(randomFrom(DataFrameTransformTaskState.values()),
|
||||
randomFrom(IndexerState.values()),
|
||||
randomBoolean() ? null : DataFrameIndexerPositionTests.randomDataFrameIndexerPosition(),
|
||||
randomLongBetween(0,10),
|
||||
randomBoolean() ? null : randomAlphaOfLength(10),
|
||||
randomBoolean() ? null : DataFrameTransformProgressTests.randomInstance(),
|
||||
randomBoolean() ? null : NodeAttributesTests.createRandom());
|
||||
}
|
||||
|
||||
public static void toXContent(DataFrameTransformState state, XContentBuilder builder) throws IOException {
|
||||
builder.startObject();
|
||||
builder.field("task_state", state.getTaskState().value());
|
||||
builder.field("indexer_state", state.getIndexerState().value());
|
||||
if (state.getPosition() != null) {
|
||||
builder.field("position");
|
||||
DataFrameIndexerPositionTests.toXContent(state.getPosition(), builder);
|
||||
}
|
||||
builder.field("checkpoint", state.getCheckpoint());
|
||||
if (state.getReason() != null) {
|
||||
builder.field("reason", state.getReason());
|
||||
}
|
||||
if (state.getProgress() != null) {
|
||||
builder.field("progress");
|
||||
DataFrameTransformProgressTests.toXContent(state.getProgress(), builder);
|
||||
}
|
||||
if (state.getNode() != null) {
|
||||
builder.field("node");
|
||||
state.getNode().toXContent(builder, ToXContent.EMPTY_PARAMS);
|
||||
}
|
||||
builder.endObject();
|
||||
}
|
||||
|
||||
}
|
@ -0,0 +1,73 @@
|
||||
/*
|
||||
* Licensed to Elasticsearch under one or more contributor
|
||||
* license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright
|
||||
* ownership. Elasticsearch licenses this file to you under
|
||||
* the Apache License, Version 2.0 (the "License"); you may
|
||||
* not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.client.dataframe.transforms;
|
||||
|
||||
import org.elasticsearch.common.xcontent.ToXContent;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import static org.elasticsearch.test.AbstractXContentTestCase.xContentTester;
|
||||
|
||||
public class DataFrameTransformStatsTests extends ESTestCase {
|
||||
|
||||
public void testFromXContent() throws IOException {
|
||||
xContentTester(this::createParser,
|
||||
DataFrameTransformStatsTests::randomInstance,
|
||||
DataFrameTransformStatsTests::toXContent,
|
||||
DataFrameTransformStats::fromXContent)
|
||||
.supportsUnknownFields(true)
|
||||
.randomFieldsExcludeFilter(field -> field.equals("node.attributes") || field.contains("position"))
|
||||
.test();
|
||||
}
|
||||
|
||||
public static DataFrameTransformStats randomInstance() {
|
||||
return new DataFrameTransformStats(randomAlphaOfLength(10),
|
||||
randomBoolean() ? null : randomFrom(DataFrameTransformTaskState.values()),
|
||||
randomBoolean() ? null : randomAlphaOfLength(100),
|
||||
randomBoolean() ? null : NodeAttributesTests.createRandom(),
|
||||
DataFrameIndexerTransformStatsTests.randomStats(),
|
||||
randomBoolean() ? null : DataFrameTransformCheckpointingInfoTests.randomDataFrameTransformCheckpointingInfo());
|
||||
}
|
||||
|
||||
public static void toXContent(DataFrameTransformStats stats, XContentBuilder builder) throws IOException {
|
||||
builder.startObject();
|
||||
builder.field(DataFrameTransformStats.ID.getPreferredName(), stats.getId());
|
||||
if (stats.getTaskState() != null) {
|
||||
builder.field(DataFrameTransformStats.TASK_STATE_FIELD.getPreferredName(),
|
||||
stats.getTaskState().value());
|
||||
}
|
||||
if (stats.getReason() != null) {
|
||||
builder.field(DataFrameTransformStats.REASON_FIELD.getPreferredName(), stats.getReason());
|
||||
}
|
||||
if (stats.getNode() != null) {
|
||||
builder.field(DataFrameTransformStats.NODE_FIELD.getPreferredName());
|
||||
stats.getNode().toXContent(builder, ToXContent.EMPTY_PARAMS);
|
||||
}
|
||||
builder.field(DataFrameTransformStats.STATS_FIELD.getPreferredName());
|
||||
DataFrameIndexerTransformStatsTests.toXContent(stats.getIndexerStats(), builder);
|
||||
if (stats.getCheckpointingInfo() != null) {
|
||||
builder.field(DataFrameTransformStats.CHECKPOINTING_INFO_FIELD.getPreferredName());
|
||||
DataFrameTransformCheckpointingInfoTests.toXContent(stats.getCheckpointingInfo(), builder);
|
||||
}
|
||||
builder.endObject();
|
||||
}
|
||||
}
|
@ -40,9 +40,13 @@ public class DataFrameIndexerPositionTests extends AbstractResponseTestCase<
|
||||
return new DataFrameIndexerPosition(instance.getIndexerPosition(), instance.getBucketsPosition());
|
||||
}
|
||||
|
||||
public static DataFrameIndexerPosition randomDataFrameIndexerPosition() {
|
||||
return new DataFrameIndexerPosition(randomPositionMap(), randomPositionMap());
|
||||
}
|
||||
|
||||
@Override
|
||||
protected DataFrameIndexerPosition createServerTestInstance() {
|
||||
return new DataFrameIndexerPosition(randomPositionMap(), randomPositionMap());
|
||||
return randomDataFrameIndexerPosition();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -48,9 +48,16 @@ public class DataFrameIndexerTransformStatsTests extends AbstractHlrcXContentTes
|
||||
return fromHlrc(instance);
|
||||
}
|
||||
|
||||
public static DataFrameIndexerTransformStats randomStats(String transformId) {
|
||||
return new DataFrameIndexerTransformStats(transformId, randomLongBetween(10L, 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));
|
||||
}
|
||||
|
||||
@Override
|
||||
protected DataFrameIndexerTransformStats createTestInstance() {
|
||||
return DataFrameTransformStateTests.randomStats(DataFrameIndexerTransformStats.DEFAULT_TRANSFORM_ID);
|
||||
return randomStats(DataFrameIndexerTransformStats.DEFAULT_TRANSFORM_ID);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -22,8 +22,10 @@ package org.elasticsearch.client.dataframe.transforms.hlrc;
|
||||
import org.elasticsearch.common.xcontent.XContentParser;
|
||||
import org.elasticsearch.client.AbstractHlrcXContentTestCase;
|
||||
import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformCheckpointStats;
|
||||
import org.elasticsearch.xpack.core.indexing.IndexerState;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.function.Predicate;
|
||||
|
||||
public class DataFrameTransformCheckpointStatsTests extends AbstractHlrcXContentTestCase<
|
||||
DataFrameTransformCheckpointStats,
|
||||
@ -31,7 +33,12 @@ public class DataFrameTransformCheckpointStatsTests extends AbstractHlrcXContent
|
||||
|
||||
public static DataFrameTransformCheckpointStats fromHlrc(
|
||||
org.elasticsearch.client.dataframe.transforms.DataFrameTransformCheckpointStats instance) {
|
||||
return new DataFrameTransformCheckpointStats(instance.getTimestampMillis(), instance.getTimeUpperBoundMillis());
|
||||
return new DataFrameTransformCheckpointStats(instance.getCheckpoint(),
|
||||
(instance.getIndexerState() != null) ? IndexerState.fromString(instance.getIndexerState().value()) : null,
|
||||
DataFrameIndexerPositionTests.fromHlrc(instance.getPosition()),
|
||||
DataFrameTransformProgressTests.fromHlrc(instance.getCheckpointProgress()),
|
||||
instance.getTimestampMillis(),
|
||||
instance.getTimeUpperBoundMillis());
|
||||
}
|
||||
|
||||
@Override
|
||||
@ -46,9 +53,17 @@ public class DataFrameTransformCheckpointStatsTests extends AbstractHlrcXContent
|
||||
return fromHlrc(instance);
|
||||
}
|
||||
|
||||
public static DataFrameTransformCheckpointStats randomDataFrameTransformCheckpointStats() {
|
||||
return new DataFrameTransformCheckpointStats(randomLongBetween(1, 1_000_000),
|
||||
randomBoolean() ? null : randomFrom(IndexerState.values()),
|
||||
DataFrameIndexerPositionTests.randomDataFrameIndexerPosition(),
|
||||
randomBoolean() ? null : DataFrameTransformProgressTests.randomDataFrameTransformProgress(),
|
||||
randomLongBetween(1, 1_000_000), randomLongBetween(0, 1_000_000));
|
||||
}
|
||||
|
||||
@Override
|
||||
protected DataFrameTransformCheckpointStats createTestInstance() {
|
||||
return DataFrameTransformStateTests.randomDataFrameTransformCheckpointStats();
|
||||
return DataFrameTransformCheckpointStatsTests.randomDataFrameTransformCheckpointStats();
|
||||
}
|
||||
|
||||
@Override
|
||||
@ -61,4 +76,8 @@ public class DataFrameTransformCheckpointStatsTests extends AbstractHlrcXContent
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Predicate<String> getRandomFieldsExcludeFilter() {
|
||||
return field -> field.startsWith("position");
|
||||
}
|
||||
}
|
||||
|
@ -24,6 +24,7 @@ import org.elasticsearch.client.AbstractHlrcXContentTestCase;
|
||||
import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformCheckpointingInfo;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.function.Predicate;
|
||||
|
||||
public class DataFrameTransformCheckpointingInfoTests extends AbstractHlrcXContentTestCase<
|
||||
DataFrameTransformCheckpointingInfo,
|
||||
@ -32,14 +33,13 @@ public class DataFrameTransformCheckpointingInfoTests extends AbstractHlrcXConte
|
||||
public static DataFrameTransformCheckpointingInfo fromHlrc(
|
||||
org.elasticsearch.client.dataframe.transforms.DataFrameTransformCheckpointingInfo instance) {
|
||||
return new DataFrameTransformCheckpointingInfo(
|
||||
DataFrameTransformCheckpointStatsTests.fromHlrc(instance.getCurrent()),
|
||||
DataFrameTransformCheckpointStatsTests.fromHlrc(instance.getInProgress()),
|
||||
DataFrameTransformCheckpointStatsTests.fromHlrc(instance.getLast()),
|
||||
DataFrameTransformCheckpointStatsTests.fromHlrc(instance.getNext()),
|
||||
instance.getOperationsBehind());
|
||||
}
|
||||
|
||||
@Override
|
||||
public org.elasticsearch.client.dataframe.transforms.DataFrameTransformCheckpointingInfo doHlrcParseInstance(XContentParser parser)
|
||||
throws IOException {
|
||||
public org.elasticsearch.client.dataframe.transforms.DataFrameTransformCheckpointingInfo doHlrcParseInstance(XContentParser parser) {
|
||||
return org.elasticsearch.client.dataframe.transforms.DataFrameTransformCheckpointingInfo.fromXContent(parser);
|
||||
}
|
||||
|
||||
@ -49,9 +49,14 @@ public class DataFrameTransformCheckpointingInfoTests extends AbstractHlrcXConte
|
||||
return fromHlrc(instance);
|
||||
}
|
||||
|
||||
public static DataFrameTransformCheckpointingInfo randomDataFrameTransformCheckpointingInfo() {
|
||||
return new DataFrameTransformCheckpointingInfo(DataFrameTransformCheckpointStatsTests.randomDataFrameTransformCheckpointStats(),
|
||||
DataFrameTransformCheckpointStatsTests.randomDataFrameTransformCheckpointStats(), randomNonNegativeLong());
|
||||
}
|
||||
|
||||
@Override
|
||||
protected DataFrameTransformCheckpointingInfo createTestInstance() {
|
||||
return DataFrameTransformStateTests.randomDataFrameTransformCheckpointingInfo();
|
||||
return randomDataFrameTransformCheckpointingInfo();
|
||||
}
|
||||
|
||||
@Override
|
||||
@ -64,4 +69,8 @@ public class DataFrameTransformCheckpointingInfoTests extends AbstractHlrcXConte
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Predicate<String> getRandomFieldsExcludeFilter() {
|
||||
return field -> field.contains("position");
|
||||
}
|
||||
}
|
||||
|
@ -37,9 +37,14 @@ public class DataFrameTransformProgressTests extends AbstractResponseTestCase<
|
||||
return new DataFrameTransformProgress(instance.getTotalDocs(), instance.getRemainingDocs());
|
||||
}
|
||||
|
||||
public static DataFrameTransformProgress randomDataFrameTransformProgress() {
|
||||
long totalDocs = randomNonNegativeLong();
|
||||
return new DataFrameTransformProgress(totalDocs, randomBoolean() ? null : randomLongBetween(0, totalDocs));
|
||||
}
|
||||
|
||||
@Override
|
||||
protected DataFrameTransformProgress createServerTestInstance() {
|
||||
return DataFrameTransformStateTests.randomDataFrameTransformProgress();
|
||||
return randomDataFrameTransformProgress();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -1,73 +0,0 @@
|
||||
/*
|
||||
* Licensed to Elasticsearch under one or more contributor
|
||||
* license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright
|
||||
* ownership. Elasticsearch licenses this file to you under
|
||||
* the Apache License, Version 2.0 (the "License"); you may
|
||||
* not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.client.dataframe.transforms.hlrc;
|
||||
|
||||
import org.elasticsearch.client.AbstractHlrcXContentTestCase;
|
||||
import org.elasticsearch.common.xcontent.XContentParser;
|
||||
import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameIndexerTransformStats;
|
||||
import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformStateAndStats;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.function.Predicate;
|
||||
|
||||
public class DataFrameTransformStateAndStatsTests extends AbstractHlrcXContentTestCase<DataFrameTransformStateAndStats,
|
||||
org.elasticsearch.client.dataframe.transforms.DataFrameTransformStateAndStats> {
|
||||
|
||||
@Override
|
||||
public org.elasticsearch.client.dataframe.transforms.DataFrameTransformStateAndStats doHlrcParseInstance(XContentParser parser)
|
||||
throws IOException {
|
||||
return org.elasticsearch.client.dataframe.transforms.DataFrameTransformStateAndStats.fromXContent(parser);
|
||||
}
|
||||
|
||||
@Override
|
||||
public DataFrameTransformStateAndStats convertHlrcToInternal(
|
||||
org.elasticsearch.client.dataframe.transforms.DataFrameTransformStateAndStats instance) {
|
||||
return new DataFrameTransformStateAndStats(instance.getId(),
|
||||
DataFrameTransformStateTests.fromHlrc(instance.getTransformState()),
|
||||
DataFrameIndexerTransformStatsTests.fromHlrc(instance.getTransformStats()),
|
||||
DataFrameTransformCheckpointingInfoTests.fromHlrc(instance.getCheckpointingInfo()));
|
||||
}
|
||||
|
||||
@Override
|
||||
protected DataFrameTransformStateAndStats createTestInstance() {
|
||||
// the transform id is not part of HLRC as it's only to a field for internal storage, therefore use a default id
|
||||
return DataFrameTransformStateTests
|
||||
.randomDataFrameTransformStateAndStats(DataFrameIndexerTransformStats.DEFAULT_TRANSFORM_ID);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected DataFrameTransformStateAndStats doParseInstance(XContentParser parser) throws IOException {
|
||||
return DataFrameTransformStateAndStats.PARSER.apply(parser, null);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean supportsUnknownFields() {
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Predicate<String> getRandomFieldsExcludeFilter() {
|
||||
return field -> field.equals("state.position.indexer_position") ||
|
||||
field.equals("state.position.bucket_position") ||
|
||||
field.equals("state.node") ||
|
||||
field.equals("state.node.attributes");
|
||||
}
|
||||
}
|
||||
|
@ -21,13 +21,11 @@ package org.elasticsearch.client.dataframe.transforms.hlrc;
|
||||
|
||||
import org.elasticsearch.client.AbstractHlrcXContentTestCase;
|
||||
import org.elasticsearch.common.xcontent.XContentParser;
|
||||
import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameIndexerPosition;
|
||||
import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameIndexerTransformStats;
|
||||
import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformCheckpointStats;
|
||||
import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformCheckpointingInfo;
|
||||
import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformProgress;
|
||||
import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformState;
|
||||
import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformStateAndStats;
|
||||
import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformStats;
|
||||
import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformTaskState;
|
||||
import org.elasticsearch.xpack.core.dataframe.transforms.NodeAttributes;
|
||||
import org.elasticsearch.xpack.core.indexing.IndexerState;
|
||||
@ -37,18 +35,8 @@ import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.function.Predicate;
|
||||
|
||||
public class DataFrameTransformStateTests extends AbstractHlrcXContentTestCase<DataFrameTransformState,
|
||||
org.elasticsearch.client.dataframe.transforms.DataFrameTransformState> {
|
||||
|
||||
public static DataFrameTransformState fromHlrc(org.elasticsearch.client.dataframe.transforms.DataFrameTransformState instance) {
|
||||
return new DataFrameTransformState(DataFrameTransformTaskState.fromString(instance.getTaskState().value()),
|
||||
IndexerState.fromString(instance.getIndexerState().value()),
|
||||
DataFrameIndexerPositionTests.fromHlrc(instance.getPosition()),
|
||||
instance.getCheckpoint(),
|
||||
instance.getReason(),
|
||||
DataFrameTransformProgressTests.fromHlrc(instance.getProgress()),
|
||||
fromHlrc(instance.getNode()));
|
||||
}
|
||||
public class DataFrameTransformStatsTests extends AbstractHlrcXContentTestCase<DataFrameTransformStats,
|
||||
org.elasticsearch.client.dataframe.transforms.DataFrameTransformStats> {
|
||||
|
||||
public static NodeAttributes fromHlrc(org.elasticsearch.client.dataframe.transforms.NodeAttributes attributes) {
|
||||
return attributes == null ? null : new NodeAttributes(attributes.getId(),
|
||||
@ -58,48 +46,63 @@ public class DataFrameTransformStateTests extends AbstractHlrcXContentTestCase<D
|
||||
attributes.getAttributes());
|
||||
}
|
||||
|
||||
public static DataFrameTransformStats
|
||||
fromHlrc(org.elasticsearch.client.dataframe.transforms.DataFrameTransformStats instance) {
|
||||
|
||||
return new DataFrameTransformStats(instance.getId(),
|
||||
DataFrameTransformTaskState.fromString(instance.getTaskState().value()),
|
||||
instance.getReason(),
|
||||
fromHlrc(instance.getNode()),
|
||||
DataFrameIndexerTransformStatsTests.fromHlrc(instance.getIndexerStats()),
|
||||
DataFrameTransformCheckpointingInfoTests.fromHlrc(instance.getCheckpointingInfo()));
|
||||
}
|
||||
|
||||
@Override
|
||||
public org.elasticsearch.client.dataframe.transforms.DataFrameTransformState doHlrcParseInstance(XContentParser parser)
|
||||
public org.elasticsearch.client.dataframe.transforms.DataFrameTransformStats doHlrcParseInstance(XContentParser parser)
|
||||
throws IOException {
|
||||
return org.elasticsearch.client.dataframe.transforms.DataFrameTransformState.fromXContent(parser);
|
||||
return org.elasticsearch.client.dataframe.transforms.DataFrameTransformStats.fromXContent(parser);
|
||||
}
|
||||
|
||||
@Override
|
||||
public DataFrameTransformState convertHlrcToInternal(org.elasticsearch.client.dataframe.transforms.DataFrameTransformState instance) {
|
||||
return fromHlrc(instance);
|
||||
public DataFrameTransformStats convertHlrcToInternal(
|
||||
org.elasticsearch.client.dataframe.transforms.DataFrameTransformStats instance) {
|
||||
return new DataFrameTransformStats(instance.getId(),
|
||||
DataFrameTransformTaskState.fromString(instance.getTaskState().value()),
|
||||
instance.getReason(),
|
||||
fromHlrc(instance.getNode()),
|
||||
DataFrameIndexerTransformStatsTests.fromHlrc(instance.getIndexerStats()),
|
||||
DataFrameTransformCheckpointingInfoTests.fromHlrc(instance.getCheckpointingInfo()));
|
||||
}
|
||||
|
||||
public static DataFrameTransformStats randomDataFrameTransformStats() {
|
||||
return new DataFrameTransformStats(randomAlphaOfLength(10),
|
||||
randomFrom(DataFrameTransformTaskState.values()),
|
||||
randomBoolean() ? null : randomAlphaOfLength(100),
|
||||
randomBoolean() ? null : randomNodeAttributes(),
|
||||
// TODO: remove this ID field from the server side as it's no longer needed
|
||||
randomStats("_all"),
|
||||
DataFrameTransformCheckpointingInfoTests.randomDataFrameTransformCheckpointingInfo());
|
||||
}
|
||||
|
||||
@Override
|
||||
protected DataFrameTransformState createTestInstance() {
|
||||
return randomDataFrameTransformState();
|
||||
protected DataFrameTransformStats createTestInstance() {
|
||||
return randomDataFrameTransformStats();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected DataFrameTransformState doParseInstance(XContentParser parser) throws IOException {
|
||||
return DataFrameTransformState.fromXContent(parser);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean supportsUnknownFields() {
|
||||
return true;
|
||||
protected DataFrameTransformStats doParseInstance(XContentParser parser) throws IOException {
|
||||
return DataFrameTransformStats.PARSER.apply(parser, null);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Predicate<String> getRandomFieldsExcludeFilter() {
|
||||
return field -> field.equals("position.indexer_position") ||
|
||||
field.equals("position.bucket_position") ||
|
||||
field.equals("node.attributes");
|
||||
return field -> field.contains("position") || field.equals("node.attributes");
|
||||
}
|
||||
|
||||
public static DataFrameTransformStateAndStats randomDataFrameTransformStateAndStats(String id) {
|
||||
return new DataFrameTransformStateAndStats(id,
|
||||
randomDataFrameTransformState(),
|
||||
randomStats(id),
|
||||
randomDataFrameTransformCheckpointingInfo());
|
||||
}
|
||||
|
||||
public static DataFrameIndexerPosition randomDataFrameIndexerPosition() {
|
||||
return new DataFrameIndexerPosition(randomPosition(), randomPosition());
|
||||
public static DataFrameTransformProgress randomDataFrameTransformProgress() {
|
||||
long totalDocs = randomNonNegativeLong();
|
||||
Long remainingDocs = randomBoolean() ? null : randomLongBetween(0, totalDocs);
|
||||
return new DataFrameTransformProgress(totalDocs, remainingDocs);
|
||||
}
|
||||
|
||||
public static DataFrameTransformCheckpointingInfo randomDataFrameTransformCheckpointingInfo() {
|
||||
@ -108,13 +111,11 @@ public class DataFrameTransformStateTests extends AbstractHlrcXContentTestCase<D
|
||||
}
|
||||
|
||||
public static DataFrameTransformCheckpointStats randomDataFrameTransformCheckpointStats() {
|
||||
return new DataFrameTransformCheckpointStats(randomNonNegativeLong(), randomNonNegativeLong());
|
||||
}
|
||||
|
||||
public static DataFrameTransformProgress randomDataFrameTransformProgress() {
|
||||
long totalDocs = randomNonNegativeLong();
|
||||
Long remainingDocs = randomBoolean() ? null : randomLongBetween(0, totalDocs);
|
||||
return new DataFrameTransformProgress(totalDocs, remainingDocs);
|
||||
return new DataFrameTransformCheckpointStats(randomLongBetween(1, 1_000_000),
|
||||
randomBoolean() ? null : randomFrom(IndexerState.values()),
|
||||
DataFrameIndexerPositionTests.randomDataFrameIndexerPosition(),
|
||||
randomBoolean() ? null : DataFrameTransformProgressTests.randomDataFrameTransformProgress(),
|
||||
randomLongBetween(1, 1_000_000), randomLongBetween(0, 1_000_000));
|
||||
}
|
||||
|
||||
public static NodeAttributes randomNodeAttributes() {
|
||||
@ -138,31 +139,13 @@ public class DataFrameTransformStateTests extends AbstractHlrcXContentTestCase<D
|
||||
randomLongBetween(0L, 10000L));
|
||||
}
|
||||
|
||||
public static DataFrameTransformState randomDataFrameTransformState() {
|
||||
return new DataFrameTransformState(randomFrom(DataFrameTransformTaskState.values()),
|
||||
randomFrom(IndexerState.values()),
|
||||
randomDataFrameIndexerPosition(),
|
||||
randomLongBetween(0,10),
|
||||
randomBoolean() ? null : randomAlphaOfLength(10),
|
||||
randomBoolean() ? null : randomDataFrameTransformProgress(),
|
||||
randomBoolean() ? null : randomNodeAttributes());
|
||||
@Override
|
||||
protected boolean supportsUnknownFields() {
|
||||
return true;
|
||||
}
|
||||
|
||||
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;
|
||||
@Override
|
||||
protected String[] getShuffleFieldsExceptions() {
|
||||
return new String[] { "position" };
|
||||
}
|
||||
}
|
@ -42,7 +42,7 @@ import org.elasticsearch.client.dataframe.StopDataFrameTransformResponse;
|
||||
import org.elasticsearch.client.dataframe.transforms.DataFrameIndexerTransformStats;
|
||||
import org.elasticsearch.client.dataframe.transforms.DataFrameTransformConfig;
|
||||
import org.elasticsearch.client.dataframe.transforms.DataFrameTransformProgress;
|
||||
import org.elasticsearch.client.dataframe.transforms.DataFrameTransformStateAndStats;
|
||||
import org.elasticsearch.client.dataframe.transforms.DataFrameTransformStats;
|
||||
import org.elasticsearch.client.dataframe.transforms.DataFrameTransformTaskState;
|
||||
import org.elasticsearch.client.dataframe.transforms.DestConfig;
|
||||
import org.elasticsearch.client.dataframe.transforms.NodeAttributes;
|
||||
@ -524,24 +524,25 @@ public class DataFrameTransformDocumentationIT extends ESRestHighLevelClientTest
|
||||
.getDataFrameTransformStats(request, RequestOptions.DEFAULT);
|
||||
// end::get-data-frame-transform-stats-execute
|
||||
|
||||
assertThat(response.getTransformsStateAndStats(), hasSize(1));
|
||||
assertThat(response.getTransformsStats(), hasSize(1));
|
||||
|
||||
// tag::get-data-frame-transform-stats-response
|
||||
DataFrameTransformStateAndStats stateAndStats =
|
||||
response.getTransformsStateAndStats().get(0); // <1>
|
||||
DataFrameTransformStats stateAndStatsInfo =
|
||||
response.getTransformsStats().get(0); // <1>
|
||||
DataFrameTransformTaskState taskState =
|
||||
stateAndStats.getTransformState().getTaskState(); // <2>
|
||||
stateAndStatsInfo.getTaskState(); // <2>
|
||||
IndexerState indexerState =
|
||||
stateAndStats.getTransformState().getIndexerState(); // <3>
|
||||
stateAndStatsInfo.getCheckpointingInfo()
|
||||
.getNext().getIndexerState(); // <3>
|
||||
DataFrameIndexerTransformStats transformStats =
|
||||
stateAndStats.getTransformStats(); // <4>
|
||||
stateAndStatsInfo.getIndexerStats(); // <4>
|
||||
DataFrameTransformProgress progress =
|
||||
stateAndStats.getTransformState().getProgress(); // <5>
|
||||
stateAndStatsInfo.getCheckpointingInfo()
|
||||
.getNext().getCheckpointProgress(); // <5>
|
||||
NodeAttributes node =
|
||||
stateAndStats.getTransformState().getNode(); // <6>
|
||||
stateAndStatsInfo.getNode(); // <6>
|
||||
// end::get-data-frame-transform-stats-response
|
||||
|
||||
assertEquals(IndexerState.STOPPED, indexerState);
|
||||
assertEquals(DataFrameTransformTaskState.STOPPED, taskState);
|
||||
assertNotNull(transformStats);
|
||||
assertNull(progress);
|
||||
|
@ -47,10 +47,10 @@ The returned +{response}+ contains the requested {dataframe-transform} statistic
|
||||
--------------------------------------------------
|
||||
include-tagged::{doc-tests-file}[{api}-response]
|
||||
--------------------------------------------------
|
||||
<1> The response contains a list of `DataFrameTransformStateAndStats` objects
|
||||
<1> The response contains a list of `DataFrameTransformStats` objects
|
||||
<2> The running state of the transform task e.g `started`
|
||||
<3> The running state of the transform indexer e.g `started`, `indexing`, etc.
|
||||
<4> The overall transform statistics recording the number of documents indexed etc.
|
||||
<5> The progress of the current run in the transform. Supplies the number of docs left until the next checkpoint
|
||||
and the total number of docs expected.
|
||||
<6> The assigned node information if the task is currently assigned to a node and running.
|
||||
<6> The assigned node information if the task is currently assigned to a node and running.
|
||||
|
@ -126,16 +126,7 @@ The API returns the following results:
|
||||
"transforms" : [
|
||||
{
|
||||
"id" : "ecommerce_transform",
|
||||
"state" : {
|
||||
"task_state" : "started",
|
||||
"indexer_state" : "started",
|
||||
"checkpoint" : 1,
|
||||
"progress" : {
|
||||
"total_docs" : 1220,
|
||||
"docs_remaining" : 0,
|
||||
"percent_complete" : 100.0
|
||||
}
|
||||
},
|
||||
"task_state" : "started",
|
||||
"stats" : {
|
||||
"pages_processed" : 2,
|
||||
"documents_processed" : 1220,
|
||||
@ -149,10 +140,31 @@ The API returns the following results:
|
||||
"search_failures" : 0
|
||||
},
|
||||
"checkpointing" : {
|
||||
"current" : {
|
||||
"timestamp_millis" : 1557474786393
|
||||
"last" : {
|
||||
"checkpoint" : 100,
|
||||
"timestamp_millis" : 1561740252497,
|
||||
"time_upper_bound_millis" : 1561740192497
|
||||
},
|
||||
"operations_behind" : 0
|
||||
"next" : {
|
||||
"checkpoint" : 101,
|
||||
"indexer_state" : "started",
|
||||
"position" : {
|
||||
"indexer_position" : {
|
||||
"hashtag" : "abcd1234"
|
||||
},
|
||||
"buckets_position" : {
|
||||
"hashtag" : "abcd5678"
|
||||
}
|
||||
},
|
||||
"checkpoint_progress" : {
|
||||
"total_docs" : 1900883,
|
||||
"docs_remaining" : 1722762,
|
||||
"percent_complete" : 9.370434687458408
|
||||
},
|
||||
"timestamp_millis" : 1561740629172,
|
||||
"time_upper_bound_millis" : 1561740569172
|
||||
},
|
||||
"operations_behind": 27000
|
||||
}
|
||||
}
|
||||
]
|
||||
|
@ -40,6 +40,10 @@ public final class DataFrameField {
|
||||
* Fields for checkpointing
|
||||
*/
|
||||
// the timestamp of the checkpoint, mandatory
|
||||
public static final ParseField CHECKPOINT = new ParseField("checkpoint");
|
||||
public static final ParseField INDEXER_STATE = new ParseField("indexer_state");
|
||||
public static final ParseField POSITION = new ParseField("position");
|
||||
public static final ParseField CHECKPOINT_PROGRESS = new ParseField("checkpoint_progress");
|
||||
public static final ParseField TIMESTAMP_MILLIS = new ParseField("timestamp_millis");
|
||||
public static final ParseField TIMESTAMP = new ParseField("timestamp");
|
||||
// checkpoint for for time based sync
|
||||
|
@ -23,7 +23,7 @@ import org.elasticsearch.tasks.Task;
|
||||
import org.elasticsearch.xpack.core.action.util.PageParams;
|
||||
import org.elasticsearch.xpack.core.action.util.QueryPage;
|
||||
import org.elasticsearch.xpack.core.dataframe.DataFrameField;
|
||||
import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformStateAndStats;
|
||||
import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformStats;
|
||||
import org.elasticsearch.xpack.core.dataframe.utils.ExceptionsHelper;
|
||||
|
||||
import java.io.IOException;
|
||||
@ -147,51 +147,51 @@ public class GetDataFrameTransformsStatsAction extends ActionType<GetDataFrameTr
|
||||
}
|
||||
|
||||
public static class Response extends BaseTasksResponse implements ToXContentObject {
|
||||
private final QueryPage<DataFrameTransformStateAndStats> transformsStateAndStats;
|
||||
private final QueryPage<DataFrameTransformStats> transformsStats;
|
||||
|
||||
public Response(List<DataFrameTransformStateAndStats> transformStateAndStats, long count) {
|
||||
public Response(List<DataFrameTransformStats> transformStateAndStats, long count) {
|
||||
this(new QueryPage<>(transformStateAndStats, count, DataFrameField.TRANSFORMS));
|
||||
}
|
||||
|
||||
public Response(List<DataFrameTransformStateAndStats> transformStateAndStats,
|
||||
public Response(List<DataFrameTransformStats> transformStateAndStats,
|
||||
long count,
|
||||
List<TaskOperationFailure> taskFailures,
|
||||
List<? extends ElasticsearchException> nodeFailures) {
|
||||
this(new QueryPage<>(transformStateAndStats, count, DataFrameField.TRANSFORMS), taskFailures, nodeFailures);
|
||||
}
|
||||
|
||||
private Response(QueryPage<DataFrameTransformStateAndStats> transformsStateAndStats) {
|
||||
this(transformsStateAndStats, Collections.emptyList(), Collections.emptyList());
|
||||
private Response(QueryPage<DataFrameTransformStats> transformsStats) {
|
||||
this(transformsStats, Collections.emptyList(), Collections.emptyList());
|
||||
}
|
||||
|
||||
private Response(QueryPage<DataFrameTransformStateAndStats> transformsStateAndStats,
|
||||
private Response(QueryPage<DataFrameTransformStats> transformsStats,
|
||||
List<TaskOperationFailure> taskFailures,
|
||||
List<? extends ElasticsearchException> nodeFailures) {
|
||||
super(taskFailures, nodeFailures);
|
||||
this.transformsStateAndStats = ExceptionsHelper.requireNonNull(transformsStateAndStats, "transformsStateAndStats");
|
||||
this.transformsStats = ExceptionsHelper.requireNonNull(transformsStats, "transformsStats");
|
||||
}
|
||||
|
||||
public Response(StreamInput in) throws IOException {
|
||||
super(in);
|
||||
if (in.getVersion().onOrAfter(Version.V_7_3_0)) {
|
||||
transformsStateAndStats = new QueryPage<>(in, DataFrameTransformStateAndStats::new);
|
||||
transformsStats = new QueryPage<>(in, DataFrameTransformStats::new);
|
||||
} else {
|
||||
List<DataFrameTransformStateAndStats> stats = in.readList(DataFrameTransformStateAndStats::new);
|
||||
transformsStateAndStats = new QueryPage<>(stats, stats.size(), DataFrameField.TRANSFORMS);
|
||||
List<DataFrameTransformStats> stats = in.readList(DataFrameTransformStats::new);
|
||||
transformsStats = new QueryPage<>(stats, stats.size(), DataFrameField.TRANSFORMS);
|
||||
}
|
||||
}
|
||||
|
||||
public List<DataFrameTransformStateAndStats> getTransformsStateAndStats() {
|
||||
return transformsStateAndStats.results();
|
||||
public List<DataFrameTransformStats> getTransformsStats() {
|
||||
return transformsStats.results();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeTo(StreamOutput out) throws IOException {
|
||||
super.writeTo(out);
|
||||
if (out.getVersion().onOrAfter(Version.V_7_3_0)) {
|
||||
transformsStateAndStats.writeTo(out);
|
||||
transformsStats.writeTo(out);
|
||||
} else {
|
||||
out.writeList(transformsStateAndStats.results());
|
||||
out.writeList(transformsStats.results());
|
||||
}
|
||||
}
|
||||
|
||||
@ -199,14 +199,14 @@ public class GetDataFrameTransformsStatsAction extends ActionType<GetDataFrameTr
|
||||
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
|
||||
builder.startObject();
|
||||
toXContentCommon(builder, params);
|
||||
transformsStateAndStats.doXContentBody(builder, params);
|
||||
transformsStats.doXContentBody(builder, params);
|
||||
builder.endObject();
|
||||
return builder;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Objects.hash(transformsStateAndStats);
|
||||
return Objects.hash(transformsStats);
|
||||
}
|
||||
|
||||
@Override
|
||||
@ -220,7 +220,7 @@ public class GetDataFrameTransformsStatsAction extends ActionType<GetDataFrameTr
|
||||
}
|
||||
|
||||
final Response that = (Response) other;
|
||||
return Objects.equals(this.transformsStateAndStats, that.transformsStateAndStats);
|
||||
return Objects.equals(this.transformsStats, that.transformsStats);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -55,7 +55,6 @@ public class DataFrameIndexerTransformStats extends IndexerJobStats {
|
||||
LENIENT_PARSER.declareLong(constructorArg(), SEARCH_TOTAL);
|
||||
LENIENT_PARSER.declareLong(constructorArg(), INDEX_FAILURES);
|
||||
LENIENT_PARSER.declareLong(constructorArg(), SEARCH_FAILURES);
|
||||
LENIENT_PARSER.declareString(optionalConstructorArg(), DataFrameField.INDEX_DOC_TYPE);
|
||||
}
|
||||
|
||||
private final String transformId;
|
||||
@ -133,7 +132,6 @@ public class DataFrameIndexerTransformStats extends IndexerJobStats {
|
||||
throw new IllegalArgumentException("when storing transform statistics, a valid transform id must be provided");
|
||||
}
|
||||
builder.field(DataFrameField.ID.getPreferredName(), transformId);
|
||||
builder.field(DataFrameField.INDEX_DOC_TYPE.getPreferredName(), NAME);
|
||||
}
|
||||
builder.endObject();
|
||||
return builder;
|
||||
|
@ -65,16 +65,16 @@ public class DataFrameTransformCheckpoint implements Writeable, ToXContentObject
|
||||
ConstructingObjectParser<DataFrameTransformCheckpoint, Void> parser = new ConstructingObjectParser<>(NAME,
|
||||
lenient, args -> {
|
||||
String id = (String) args[0];
|
||||
Long timestamp = (Long) args[1];
|
||||
Long checkpoint = (Long) args[2];
|
||||
long timestamp = (Long) args[1];
|
||||
long checkpoint = (Long) args[2];
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
Map<String, long[]> checkpoints = (Map<String, long[]>) args[3];
|
||||
|
||||
Long timestamp_checkpoint = (Long) args[4];
|
||||
Long timeUpperBound = (Long) args[4];
|
||||
|
||||
// ignored, only for internal storage: String docType = (String) args[5];
|
||||
return new DataFrameTransformCheckpoint(id, timestamp, checkpoint, checkpoints, timestamp_checkpoint);
|
||||
return new DataFrameTransformCheckpoint(id, timestamp, checkpoint, checkpoints, timeUpperBound);
|
||||
});
|
||||
|
||||
parser.declareString(constructorArg(), DataFrameField.ID);
|
||||
@ -108,13 +108,13 @@ public class DataFrameTransformCheckpoint implements Writeable, ToXContentObject
|
||||
return parser;
|
||||
}
|
||||
|
||||
public DataFrameTransformCheckpoint(String transformId, Long timestamp, Long checkpoint, Map<String, long[]> checkpoints,
|
||||
public DataFrameTransformCheckpoint(String transformId, long timestamp, long checkpoint, Map<String, long[]> checkpoints,
|
||||
Long timeUpperBound) {
|
||||
this.transformId = transformId;
|
||||
this.timestampMillis = timestamp.longValue();
|
||||
this.transformId = Objects.requireNonNull(transformId);
|
||||
this.timestampMillis = timestamp;
|
||||
this.checkpoint = checkpoint;
|
||||
this.indicesCheckpoints = Collections.unmodifiableMap(checkpoints);
|
||||
this.timeUpperBoundMillis = timeUpperBound == null ? 0 : timeUpperBound.longValue();
|
||||
this.timeUpperBoundMillis = timeUpperBound == null ? 0 : timeUpperBound;
|
||||
}
|
||||
|
||||
public DataFrameTransformCheckpoint(StreamInput in) throws IOException {
|
||||
|
@ -6,18 +6,23 @@
|
||||
|
||||
package org.elasticsearch.xpack.core.dataframe.transforms;
|
||||
|
||||
import org.elasticsearch.Version;
|
||||
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.ObjectParser;
|
||||
import org.elasticsearch.common.xcontent.ToXContentObject;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.common.xcontent.XContentParser;
|
||||
import org.elasticsearch.xpack.core.dataframe.DataFrameField;
|
||||
import org.elasticsearch.xpack.core.indexing.IndexerState;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Objects;
|
||||
|
||||
import static org.elasticsearch.common.xcontent.ConstructingObjectParser.optionalConstructorArg;
|
||||
|
||||
/**
|
||||
* Checkpoint stats data for 1 checkpoint
|
||||
*
|
||||
@ -25,34 +30,92 @@ import java.util.Objects;
|
||||
*/
|
||||
public class DataFrameTransformCheckpointStats implements Writeable, ToXContentObject {
|
||||
|
||||
public static DataFrameTransformCheckpointStats EMPTY = new DataFrameTransformCheckpointStats(0L, 0L);
|
||||
public static final DataFrameTransformCheckpointStats EMPTY = new DataFrameTransformCheckpointStats(0L, null, null, null, 0L, 0L);
|
||||
|
||||
private final long checkpoint;
|
||||
private final IndexerState indexerState;
|
||||
private final DataFrameIndexerPosition position;
|
||||
private final DataFrameTransformProgress checkpointProgress;
|
||||
private final long timestampMillis;
|
||||
private final long timeUpperBoundMillis;
|
||||
|
||||
private static final ConstructingObjectParser<DataFrameTransformCheckpointStats, Void> LENIENT_PARSER = new ConstructingObjectParser<>(
|
||||
static final ConstructingObjectParser<DataFrameTransformCheckpointStats, Void> LENIENT_PARSER = new ConstructingObjectParser<>(
|
||||
"data_frame_transform_checkpoint_stats", true, args -> {
|
||||
long timestamp = args[0] == null ? 0L : (Long) args[0];
|
||||
long timeUpperBound = args[1] == null ? 0L : (Long) args[1];
|
||||
long checkpoint = args[0] == null ? 0L : (Long) args[0];
|
||||
IndexerState indexerState = (IndexerState) args[1];
|
||||
DataFrameIndexerPosition position = (DataFrameIndexerPosition) args[2];
|
||||
DataFrameTransformProgress checkpointProgress = (DataFrameTransformProgress) args[3];
|
||||
long timestamp = args[4] == null ? 0L : (Long) args[4];
|
||||
long timeUpperBound = args[5] == null ? 0L : (Long) args[5];
|
||||
|
||||
return new DataFrameTransformCheckpointStats(timestamp, timeUpperBound);
|
||||
});
|
||||
return new DataFrameTransformCheckpointStats(checkpoint, indexerState, position, checkpointProgress, timestamp, timeUpperBound);
|
||||
});
|
||||
|
||||
static {
|
||||
LENIENT_PARSER.declareLong(ConstructingObjectParser.optionalConstructorArg(), DataFrameField.TIMESTAMP_MILLIS);
|
||||
LENIENT_PARSER.declareLong(ConstructingObjectParser.optionalConstructorArg(), DataFrameField.TIME_UPPER_BOUND_MILLIS);
|
||||
LENIENT_PARSER.declareLong(optionalConstructorArg(), DataFrameField.CHECKPOINT);
|
||||
LENIENT_PARSER.declareField(optionalConstructorArg(), p -> IndexerState.fromString(p.text()), DataFrameField.INDEXER_STATE,
|
||||
ObjectParser.ValueType.STRING);
|
||||
LENIENT_PARSER.declareObject(optionalConstructorArg(), DataFrameIndexerPosition.PARSER, DataFrameField.POSITION);
|
||||
LENIENT_PARSER.declareObject(optionalConstructorArg(), DataFrameTransformProgress.PARSER, DataFrameField.CHECKPOINT_PROGRESS);
|
||||
LENIENT_PARSER.declareLong(optionalConstructorArg(), DataFrameField.TIMESTAMP_MILLIS);
|
||||
LENIENT_PARSER.declareLong(optionalConstructorArg(), DataFrameField.TIME_UPPER_BOUND_MILLIS);
|
||||
}
|
||||
|
||||
public DataFrameTransformCheckpointStats(final long timestampMillis, final long timeUpperBoundMillis) {
|
||||
public DataFrameTransformCheckpointStats(final long checkpoint, final IndexerState indexerState,
|
||||
final DataFrameIndexerPosition position, final DataFrameTransformProgress checkpointProgress,
|
||||
final long timestampMillis, final long timeUpperBoundMillis) {
|
||||
this.checkpoint = checkpoint;
|
||||
this.indexerState = indexerState;
|
||||
this.position = position;
|
||||
this.checkpointProgress = checkpointProgress;
|
||||
this.timestampMillis = timestampMillis;
|
||||
this.timeUpperBoundMillis = timeUpperBoundMillis;
|
||||
}
|
||||
|
||||
public DataFrameTransformCheckpointStats(StreamInput in) throws IOException {
|
||||
if (in.getVersion().onOrAfter(Version.V_7_4_0)) {
|
||||
this.checkpoint = in.readVLong();
|
||||
if (in.readBoolean()) {
|
||||
this.indexerState = in.readEnum(IndexerState.class);
|
||||
} else {
|
||||
this.indexerState = null;
|
||||
}
|
||||
if (in.readBoolean()) {
|
||||
this.position = new DataFrameIndexerPosition(in);
|
||||
} else {
|
||||
this.position = null;
|
||||
}
|
||||
if (in.readBoolean()) {
|
||||
this.checkpointProgress = new DataFrameTransformProgress(in);
|
||||
} else {
|
||||
this.checkpointProgress = null;
|
||||
}
|
||||
} else {
|
||||
this.checkpoint = 0;
|
||||
this.indexerState = null;
|
||||
this.position = null;
|
||||
this.checkpointProgress = null;
|
||||
}
|
||||
this.timestampMillis = in.readLong();
|
||||
this.timeUpperBoundMillis = in.readLong();
|
||||
}
|
||||
|
||||
public long getCheckpoint() {
|
||||
return checkpoint;
|
||||
}
|
||||
|
||||
public IndexerState getIndexerState() {
|
||||
return indexerState;
|
||||
}
|
||||
|
||||
public DataFrameIndexerPosition getPosition() {
|
||||
return position;
|
||||
}
|
||||
|
||||
public DataFrameTransformProgress getCheckpointProgress() {
|
||||
return checkpointProgress;
|
||||
}
|
||||
|
||||
public long getTimestampMillis() {
|
||||
return timestampMillis;
|
||||
}
|
||||
@ -64,11 +127,23 @@ public class DataFrameTransformCheckpointStats implements Writeable, ToXContentO
|
||||
@Override
|
||||
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
|
||||
builder.startObject();
|
||||
builder.timeField(DataFrameField.TIMESTAMP_MILLIS.getPreferredName(), DataFrameField.TIMESTAMP.getPreferredName(),
|
||||
getTimestampMillis());
|
||||
builder.field(DataFrameField.CHECKPOINT.getPreferredName(), checkpoint);
|
||||
if (indexerState != null) {
|
||||
builder.field(DataFrameField.INDEXER_STATE.getPreferredName(), indexerState.value());
|
||||
}
|
||||
if (position != null) {
|
||||
builder.field(DataFrameField.POSITION.getPreferredName(), position);
|
||||
}
|
||||
if (checkpointProgress != null) {
|
||||
builder.field(DataFrameField.CHECKPOINT_PROGRESS.getPreferredName(), checkpointProgress);
|
||||
}
|
||||
if (timestampMillis > 0) {
|
||||
builder.timeField(DataFrameField.TIMESTAMP_MILLIS.getPreferredName(), DataFrameField.TIMESTAMP.getPreferredName(),
|
||||
timestampMillis);
|
||||
}
|
||||
if (timeUpperBoundMillis > 0) {
|
||||
builder.timeField(DataFrameField.TIME_UPPER_BOUND_MILLIS.getPreferredName(), DataFrameField.TIME_UPPER_BOUND.getPreferredName(),
|
||||
timeUpperBoundMillis);
|
||||
timeUpperBoundMillis);
|
||||
}
|
||||
builder.endObject();
|
||||
return builder;
|
||||
@ -76,13 +151,34 @@ public class DataFrameTransformCheckpointStats implements Writeable, ToXContentO
|
||||
|
||||
@Override
|
||||
public void writeTo(StreamOutput out) throws IOException {
|
||||
if (out.getVersion().onOrAfter(Version.V_7_4_0)) {
|
||||
out.writeVLong(checkpoint);
|
||||
if (indexerState != null) {
|
||||
out.writeBoolean(true);
|
||||
out.writeEnum(indexerState);
|
||||
} else {
|
||||
out.writeBoolean(false);
|
||||
}
|
||||
if (position != null) {
|
||||
out.writeBoolean(true);
|
||||
position.writeTo(out);
|
||||
} else {
|
||||
out.writeBoolean(false);
|
||||
}
|
||||
if (checkpointProgress != null) {
|
||||
out.writeBoolean(true);
|
||||
checkpointProgress.writeTo(out);
|
||||
} else {
|
||||
out.writeBoolean(false);
|
||||
}
|
||||
}
|
||||
out.writeLong(timestampMillis);
|
||||
out.writeLong(timeUpperBoundMillis);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Objects.hash(timestampMillis, timeUpperBoundMillis);
|
||||
return Objects.hash(checkpoint, indexerState, position, checkpointProgress, timestampMillis, timeUpperBoundMillis);
|
||||
}
|
||||
|
||||
@Override
|
||||
@ -97,12 +193,15 @@ public class DataFrameTransformCheckpointStats implements Writeable, ToXContentO
|
||||
|
||||
DataFrameTransformCheckpointStats that = (DataFrameTransformCheckpointStats) other;
|
||||
|
||||
return this.timestampMillis == that.timestampMillis &&
|
||||
this.timeUpperBoundMillis == that.timeUpperBoundMillis;
|
||||
return this.checkpoint == that.checkpoint
|
||||
&& Objects.equals(this.indexerState, that.indexerState)
|
||||
&& Objects.equals(this.position, that.position)
|
||||
&& Objects.equals(this.checkpointProgress, that.checkpointProgress)
|
||||
&& this.timestampMillis == that.timestampMillis
|
||||
&& this.timeUpperBoundMillis == that.timeUpperBoundMillis;
|
||||
}
|
||||
|
||||
public static DataFrameTransformCheckpointStats fromXContent(XContentParser p) {
|
||||
return LENIENT_PARSER.apply(p, null);
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
@ -7,6 +7,7 @@
|
||||
package org.elasticsearch.xpack.core.dataframe.transforms;
|
||||
|
||||
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;
|
||||
@ -26,17 +27,17 @@ import java.util.Objects;
|
||||
*/
|
||||
public class DataFrameTransformCheckpointingInfo implements Writeable, ToXContentObject {
|
||||
|
||||
public static DataFrameTransformCheckpointingInfo EMPTY = new DataFrameTransformCheckpointingInfo(
|
||||
public static final DataFrameTransformCheckpointingInfo EMPTY = new DataFrameTransformCheckpointingInfo(
|
||||
DataFrameTransformCheckpointStats.EMPTY,
|
||||
DataFrameTransformCheckpointStats.EMPTY,
|
||||
0L);
|
||||
|
||||
public static final ParseField CURRENT_CHECKPOINT = new ParseField("current");
|
||||
public static final ParseField IN_PROGRESS_CHECKPOINT = new ParseField("in_progress");
|
||||
public static final ParseField LAST_CHECKPOINT = new ParseField("last");
|
||||
public static final ParseField NEXT_CHECKPOINT = new ParseField("next");
|
||||
public static final ParseField OPERATIONS_BEHIND = new ParseField("operations_behind");
|
||||
|
||||
private final DataFrameTransformCheckpointStats current;
|
||||
private final DataFrameTransformCheckpointStats inProgress;
|
||||
private final DataFrameTransformCheckpointStats last;
|
||||
private final DataFrameTransformCheckpointStats next;
|
||||
private final long operationsBehind;
|
||||
|
||||
private static final ConstructingObjectParser<DataFrameTransformCheckpointingInfo, Void> LENIENT_PARSER =
|
||||
@ -51,39 +52,39 @@ public class DataFrameTransformCheckpointingInfo implements Writeable, ToXConten
|
||||
|
||||
static {
|
||||
LENIENT_PARSER.declareObject(ConstructingObjectParser.optionalConstructorArg(),
|
||||
(p, c) -> DataFrameTransformCheckpointStats.fromXContent(p), CURRENT_CHECKPOINT);
|
||||
DataFrameTransformCheckpointStats.LENIENT_PARSER::apply, LAST_CHECKPOINT);
|
||||
LENIENT_PARSER.declareObject(ConstructingObjectParser.optionalConstructorArg(),
|
||||
(p, c) -> DataFrameTransformCheckpointStats.fromXContent(p), IN_PROGRESS_CHECKPOINT);
|
||||
DataFrameTransformCheckpointStats.LENIENT_PARSER::apply, NEXT_CHECKPOINT);
|
||||
LENIENT_PARSER.declareLong(ConstructingObjectParser.optionalConstructorArg(), OPERATIONS_BEHIND);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create checkpoint stats object with checkpoint information about the current and in progress checkpoint as well as the current state
|
||||
* Create checkpoint stats object with checkpoint information about the last and next checkpoint as well as the current state
|
||||
* of source.
|
||||
*
|
||||
* @param current stats of the current checkpoint
|
||||
* @param inProgress stats of the in progress checkpoint
|
||||
* @param last stats of the last checkpoint
|
||||
* @param next stats of the next checkpoint
|
||||
* @param operationsBehind counter of operations the current checkpoint is behind source
|
||||
*/
|
||||
public DataFrameTransformCheckpointingInfo(DataFrameTransformCheckpointStats current, DataFrameTransformCheckpointStats inProgress,
|
||||
public DataFrameTransformCheckpointingInfo(DataFrameTransformCheckpointStats last, DataFrameTransformCheckpointStats next,
|
||||
long operationsBehind) {
|
||||
this.current = Objects.requireNonNull(current);
|
||||
this.inProgress = Objects.requireNonNull(inProgress);
|
||||
this.last = Objects.requireNonNull(last);
|
||||
this.next = Objects.requireNonNull(next);
|
||||
this.operationsBehind = operationsBehind;
|
||||
}
|
||||
|
||||
public DataFrameTransformCheckpointingInfo(StreamInput in) throws IOException {
|
||||
current = new DataFrameTransformCheckpointStats(in);
|
||||
inProgress = new DataFrameTransformCheckpointStats(in);
|
||||
last = new DataFrameTransformCheckpointStats(in);
|
||||
next = new DataFrameTransformCheckpointStats(in);
|
||||
operationsBehind = in.readLong();
|
||||
}
|
||||
|
||||
public DataFrameTransformCheckpointStats getCurrent() {
|
||||
return current;
|
||||
public DataFrameTransformCheckpointStats getLast() {
|
||||
return last;
|
||||
}
|
||||
|
||||
public DataFrameTransformCheckpointStats getInProgress() {
|
||||
return inProgress;
|
||||
public DataFrameTransformCheckpointStats getNext() {
|
||||
return next;
|
||||
}
|
||||
|
||||
public long getOperationsBehind() {
|
||||
@ -93,13 +94,10 @@ public class DataFrameTransformCheckpointingInfo implements Writeable, ToXConten
|
||||
@Override
|
||||
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
|
||||
builder.startObject();
|
||||
if (current.getTimestampMillis() > 0) {
|
||||
builder.field(CURRENT_CHECKPOINT.getPreferredName(), current);
|
||||
builder.field(LAST_CHECKPOINT.getPreferredName(), last);
|
||||
if (next.getCheckpoint() > 0) {
|
||||
builder.field(NEXT_CHECKPOINT.getPreferredName(), next);
|
||||
}
|
||||
if (inProgress.getTimestampMillis() > 0) {
|
||||
builder.field(IN_PROGRESS_CHECKPOINT.getPreferredName(), inProgress);
|
||||
}
|
||||
|
||||
builder.field(OPERATIONS_BEHIND.getPreferredName(), operationsBehind);
|
||||
builder.endObject();
|
||||
return builder;
|
||||
@ -107,8 +105,8 @@ public class DataFrameTransformCheckpointingInfo implements Writeable, ToXConten
|
||||
|
||||
@Override
|
||||
public void writeTo(StreamOutput out) throws IOException {
|
||||
current.writeTo(out);
|
||||
inProgress.writeTo(out);
|
||||
last.writeTo(out);
|
||||
next.writeTo(out);
|
||||
out.writeLong(operationsBehind);
|
||||
}
|
||||
|
||||
@ -118,7 +116,7 @@ public class DataFrameTransformCheckpointingInfo implements Writeable, ToXConten
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Objects.hash(current, inProgress, operationsBehind);
|
||||
return Objects.hash(last, next, operationsBehind);
|
||||
}
|
||||
|
||||
@Override
|
||||
@ -133,9 +131,13 @@ public class DataFrameTransformCheckpointingInfo implements Writeable, ToXConten
|
||||
|
||||
DataFrameTransformCheckpointingInfo that = (DataFrameTransformCheckpointingInfo) other;
|
||||
|
||||
return Objects.equals(this.current, that.current) &&
|
||||
Objects.equals(this.inProgress, that.inProgress) &&
|
||||
return Objects.equals(this.last, that.last) &&
|
||||
Objects.equals(this.next, that.next) &&
|
||||
this.operationsBehind == that.operationsBehind;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return Strings.toString(this);
|
||||
}
|
||||
}
|
||||
|
@ -0,0 +1,236 @@
|
||||
/*
|
||||
* 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.transforms;
|
||||
|
||||
import org.elasticsearch.Version;
|
||||
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.ObjectParser;
|
||||
import org.elasticsearch.common.xcontent.ToXContentObject;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.common.xcontent.XContentParser;
|
||||
import org.elasticsearch.xpack.core.dataframe.DataFrameField;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Objects;
|
||||
|
||||
import static org.elasticsearch.common.xcontent.ConstructingObjectParser.constructorArg;
|
||||
import static org.elasticsearch.common.xcontent.ConstructingObjectParser.optionalConstructorArg;
|
||||
|
||||
/**
|
||||
* Used as a wrapper for the objects returned from the stats endpoint.
|
||||
* Objects of this class are expected to be ephemeral.
|
||||
* Do not persist objects of this class to cluster state or an index.
|
||||
*/
|
||||
public class DataFrameTransformStats implements Writeable, ToXContentObject {
|
||||
|
||||
public static final String NAME = "data_frame_transform_stats";
|
||||
public static final ParseField TASK_STATE_FIELD = new ParseField("task_state");
|
||||
public static final ParseField REASON_FIELD = new ParseField("reason");
|
||||
public static final ParseField NODE_FIELD = new ParseField("node");
|
||||
public static final ParseField CHECKPOINTING_INFO_FIELD = new ParseField("checkpointing");
|
||||
|
||||
private final String id;
|
||||
private final DataFrameTransformTaskState taskState;
|
||||
@Nullable
|
||||
private final String reason;
|
||||
@Nullable
|
||||
private NodeAttributes node;
|
||||
private final DataFrameIndexerTransformStats indexerStats;
|
||||
private final DataFrameTransformCheckpointingInfo checkpointingInfo;
|
||||
|
||||
public static final ConstructingObjectParser<DataFrameTransformStats, Void> PARSER = new ConstructingObjectParser<>(
|
||||
NAME,
|
||||
true,
|
||||
a -> new DataFrameTransformStats((String) a[0],
|
||||
(DataFrameTransformTaskState) a[1],
|
||||
(String) a[2],
|
||||
(NodeAttributes) a[3],
|
||||
(DataFrameIndexerTransformStats) a[4],
|
||||
(DataFrameTransformCheckpointingInfo) a[5]));
|
||||
|
||||
static {
|
||||
PARSER.declareString(constructorArg(), DataFrameField.ID);
|
||||
PARSER.declareField(constructorArg(), p -> DataFrameTransformTaskState.fromString(p.text()), TASK_STATE_FIELD,
|
||||
ObjectParser.ValueType.STRING);
|
||||
PARSER.declareString(optionalConstructorArg(), REASON_FIELD);
|
||||
PARSER.declareField(optionalConstructorArg(), NodeAttributes.PARSER::apply, NODE_FIELD, ObjectParser.ValueType.OBJECT);
|
||||
PARSER.declareObject(constructorArg(), (p, c) -> DataFrameIndexerTransformStats.fromXContent(p),
|
||||
DataFrameField.STATS_FIELD);
|
||||
PARSER.declareObject(constructorArg(),
|
||||
(p, c) -> DataFrameTransformCheckpointingInfo.fromXContent(p), CHECKPOINTING_INFO_FIELD);
|
||||
}
|
||||
|
||||
public static DataFrameTransformStats fromXContent(XContentParser parser) throws IOException {
|
||||
return PARSER.parse(parser, null);
|
||||
}
|
||||
|
||||
public static DataFrameTransformStats initialStats(String id) {
|
||||
return stoppedStats(id, new DataFrameIndexerTransformStats(id));
|
||||
}
|
||||
|
||||
public static DataFrameTransformStats stoppedStats(String id, DataFrameIndexerTransformStats indexerTransformStats) {
|
||||
return new DataFrameTransformStats(id,
|
||||
DataFrameTransformTaskState.STOPPED,
|
||||
null,
|
||||
null,
|
||||
indexerTransformStats,
|
||||
DataFrameTransformCheckpointingInfo.EMPTY);
|
||||
}
|
||||
|
||||
|
||||
public DataFrameTransformStats(String id, DataFrameTransformTaskState taskState, @Nullable String reason,
|
||||
@Nullable NodeAttributes node, DataFrameIndexerTransformStats stats,
|
||||
DataFrameTransformCheckpointingInfo checkpointingInfo) {
|
||||
this.id = Objects.requireNonNull(id);
|
||||
this.taskState = Objects.requireNonNull(taskState);
|
||||
this.reason = reason;
|
||||
this.node = node;
|
||||
this.indexerStats = Objects.requireNonNull(stats);
|
||||
this.checkpointingInfo = Objects.requireNonNull(checkpointingInfo);
|
||||
}
|
||||
|
||||
public DataFrameTransformStats(StreamInput in) throws IOException {
|
||||
if (in.getVersion().onOrAfter(Version.V_7_4_0)) {
|
||||
this.id = in.readString();
|
||||
this.taskState = in.readEnum(DataFrameTransformTaskState.class);
|
||||
this.reason = in.readOptionalString();
|
||||
if (in.readBoolean()) {
|
||||
this.node = new NodeAttributes(in);
|
||||
} else {
|
||||
this.node = null;
|
||||
}
|
||||
this.indexerStats = new DataFrameIndexerTransformStats(in);
|
||||
this.checkpointingInfo = new DataFrameTransformCheckpointingInfo(in);
|
||||
|
||||
} else {
|
||||
// Prior to version 7.4 DataFrameTransformStats didn't exist, and we have
|
||||
// to do the best we can of reading from a DataFrameTransformStoredDoc object
|
||||
// (which is called DataFrameTransformStateAndStats in 7.2/7.3)
|
||||
this.id = in.readString();
|
||||
DataFrameTransformState state = new DataFrameTransformState(in);
|
||||
this.taskState = state.getTaskState();
|
||||
this.reason = state.getReason();
|
||||
this.node = null;
|
||||
this.indexerStats = new DataFrameIndexerTransformStats(in);
|
||||
this.checkpointingInfo = new DataFrameTransformCheckpointingInfo(in);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
|
||||
builder.startObject();
|
||||
builder.field(DataFrameField.ID.getPreferredName(), id);
|
||||
builder.field(TASK_STATE_FIELD.getPreferredName(), taskState.value());
|
||||
if (reason != null) {
|
||||
builder.field(REASON_FIELD.getPreferredName(), reason);
|
||||
}
|
||||
if (node != null) {
|
||||
builder.field(NODE_FIELD.getPreferredName(), node);
|
||||
}
|
||||
builder.field(DataFrameField.STATS_FIELD.getPreferredName(), indexerStats, params);
|
||||
builder.field(CHECKPOINTING_INFO_FIELD.getPreferredName(), checkpointingInfo, params);
|
||||
builder.endObject();
|
||||
return builder;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeTo(StreamOutput out) throws IOException {
|
||||
if (out.getVersion().onOrAfter(Version.V_7_4_0)) {
|
||||
out.writeString(id);
|
||||
out.writeEnum(taskState);
|
||||
out.writeOptionalString(reason);
|
||||
if (node != null) {
|
||||
out.writeBoolean(true);
|
||||
node.writeTo(out);
|
||||
} else {
|
||||
out.writeBoolean(false);
|
||||
}
|
||||
indexerStats.writeTo(out);
|
||||
checkpointingInfo.writeTo(out);
|
||||
} else {
|
||||
// Prior to version 7.4 DataFrameTransformStats didn't exist, and we have
|
||||
// to do the best we can of writing to a DataFrameTransformStoredDoc object
|
||||
// (which is called DataFrameTransformStateAndStats in 7.2/7.3)
|
||||
out.writeString(id);
|
||||
new DataFrameTransformState(taskState,
|
||||
checkpointingInfo.getNext().getIndexerState(),
|
||||
checkpointingInfo.getNext().getPosition(),
|
||||
checkpointingInfo.getLast().getCheckpoint(),
|
||||
reason,
|
||||
checkpointingInfo.getNext().getCheckpointProgress()).writeTo(out);
|
||||
out.writeBoolean(false);
|
||||
indexerStats.writeTo(out);
|
||||
checkpointingInfo.writeTo(out);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Objects.hash(id, taskState, reason, node, indexerStats, checkpointingInfo);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object other) {
|
||||
if (this == other) {
|
||||
return true;
|
||||
}
|
||||
|
||||
if (other == null || getClass() != other.getClass()) {
|
||||
return false;
|
||||
}
|
||||
|
||||
DataFrameTransformStats that = (DataFrameTransformStats) other;
|
||||
|
||||
return Objects.equals(this.id, that.id)
|
||||
&& Objects.equals(this.taskState, that.taskState)
|
||||
&& Objects.equals(this.reason, that.reason)
|
||||
&& Objects.equals(this.node, that.node)
|
||||
&& Objects.equals(this.indexerStats, that.indexerStats)
|
||||
&& Objects.equals(this.checkpointingInfo, that.checkpointingInfo);
|
||||
}
|
||||
|
||||
public String getId() {
|
||||
return id;
|
||||
}
|
||||
|
||||
public DataFrameTransformTaskState getTaskState() {
|
||||
return taskState;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
public String getReason() {
|
||||
return reason;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
public NodeAttributes getNode() {
|
||||
return node;
|
||||
}
|
||||
|
||||
public void setNode(NodeAttributes node) {
|
||||
this.node = node;
|
||||
}
|
||||
|
||||
public DataFrameIndexerTransformStats getIndexerStats() {
|
||||
return indexerStats;
|
||||
}
|
||||
|
||||
public DataFrameTransformCheckpointingInfo getCheckpointingInfo() {
|
||||
return checkpointingInfo;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return Strings.toString(this);
|
||||
}
|
||||
}
|
@ -6,7 +6,7 @@
|
||||
|
||||
package org.elasticsearch.xpack.core.dataframe.transforms;
|
||||
|
||||
import org.elasticsearch.common.Nullable;
|
||||
import org.elasticsearch.Version;
|
||||
import org.elasticsearch.common.ParseField;
|
||||
import org.elasticsearch.common.Strings;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
@ -17,53 +17,40 @@ import org.elasticsearch.common.xcontent.ToXContentObject;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.common.xcontent.XContentParser;
|
||||
import org.elasticsearch.xpack.core.dataframe.DataFrameField;
|
||||
import org.elasticsearch.xpack.core.indexing.IndexerState;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Objects;
|
||||
|
||||
public class DataFrameTransformStateAndStats implements Writeable, ToXContentObject {
|
||||
/**
|
||||
* A wrapper for grouping transform state and stats when persisting to an index.
|
||||
* Not intended to be returned in endpoint responses.
|
||||
*/
|
||||
public class DataFrameTransformStoredDoc implements Writeable, ToXContentObject {
|
||||
|
||||
public static final String NAME = "data_frame_transform_state_and_stats";
|
||||
public static final ParseField STATE_FIELD = new ParseField("state");
|
||||
public static final ParseField CHECKPOINTING_INFO_FIELD = new ParseField("checkpointing");
|
||||
|
||||
private final String id;
|
||||
private final DataFrameTransformState transformState;
|
||||
private final DataFrameIndexerTransformStats transformStats;
|
||||
private final DataFrameTransformCheckpointingInfo checkpointingInfo;
|
||||
|
||||
public static final ConstructingObjectParser<DataFrameTransformStateAndStats, Void> PARSER = new ConstructingObjectParser<>(
|
||||
public static final ConstructingObjectParser<DataFrameTransformStoredDoc, Void> PARSER = new ConstructingObjectParser<>(
|
||||
NAME, true,
|
||||
a -> new DataFrameTransformStateAndStats((String) a[0],
|
||||
a -> new DataFrameTransformStoredDoc((String) a[0],
|
||||
(DataFrameTransformState) a[1],
|
||||
(DataFrameIndexerTransformStats) a[2],
|
||||
(DataFrameTransformCheckpointingInfo) a[3]));
|
||||
(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);
|
||||
PARSER.declareObject(ConstructingObjectParser.optionalConstructorArg(),
|
||||
(p, c) -> DataFrameTransformCheckpointingInfo.fromXContent(p), CHECKPOINTING_INFO_FIELD);
|
||||
}
|
||||
|
||||
public static DataFrameTransformStateAndStats fromXContent(XContentParser parser) throws IOException {
|
||||
public static DataFrameTransformStoredDoc fromXContent(XContentParser parser) throws IOException {
|
||||
return PARSER.parse(parser, null);
|
||||
}
|
||||
|
||||
public static DataFrameTransformStateAndStats initialStateAndStats(String id) {
|
||||
return initialStateAndStats(id, new DataFrameIndexerTransformStats(id));
|
||||
}
|
||||
|
||||
public static DataFrameTransformStateAndStats initialStateAndStats(String id, DataFrameIndexerTransformStats indexerTransformStats) {
|
||||
return new DataFrameTransformStateAndStats(id,
|
||||
new DataFrameTransformState(DataFrameTransformTaskState.STOPPED, IndexerState.STOPPED, null, 0L, null, null),
|
||||
indexerTransformStats,
|
||||
DataFrameTransformCheckpointingInfo.EMPTY);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the persisted state and stats document name from the Data Frame Transform Id.
|
||||
*
|
||||
@ -73,24 +60,19 @@ public class DataFrameTransformStateAndStats implements Writeable, ToXContentObj
|
||||
return NAME + "-" + transformId;
|
||||
}
|
||||
|
||||
public DataFrameTransformStateAndStats(String id, DataFrameTransformState state, DataFrameIndexerTransformStats stats,
|
||||
DataFrameTransformCheckpointingInfo checkpointingInfo) {
|
||||
public DataFrameTransformStoredDoc(String id, DataFrameTransformState state, DataFrameIndexerTransformStats stats) {
|
||||
this.id = Objects.requireNonNull(id);
|
||||
this.transformState = Objects.requireNonNull(state);
|
||||
this.transformStats = Objects.requireNonNull(stats);
|
||||
this.checkpointingInfo = Objects.requireNonNull(checkpointingInfo);
|
||||
}
|
||||
|
||||
public DataFrameTransformStateAndStats(StreamInput in) throws IOException {
|
||||
public DataFrameTransformStoredDoc(StreamInput in) throws IOException {
|
||||
this.id = in.readString();
|
||||
this.transformState = new DataFrameTransformState(in);
|
||||
this.transformStats = new DataFrameIndexerTransformStats(in);
|
||||
this.checkpointingInfo = new DataFrameTransformCheckpointingInfo(in);
|
||||
}
|
||||
|
||||
@Nullable
|
||||
public String getTransformId() {
|
||||
return transformStats.getTransformId();
|
||||
if (in.getVersion().before(Version.V_7_4_0)) {
|
||||
new DataFrameTransformCheckpointingInfo(in);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
@ -99,10 +81,7 @@ public class DataFrameTransformStateAndStats implements Writeable, ToXContentObj
|
||||
builder.field(DataFrameField.ID.getPreferredName(), id);
|
||||
builder.field(STATE_FIELD.getPreferredName(), transformState, params);
|
||||
builder.field(DataFrameField.STATS_FIELD.getPreferredName(), transformStats, params);
|
||||
builder.field(CHECKPOINTING_INFO_FIELD.getPreferredName(), checkpointingInfo, params);
|
||||
if (params.paramAsBoolean(DataFrameField.FOR_INTERNAL_STORAGE, false)) {
|
||||
builder.field(DataFrameField.INDEX_DOC_TYPE.getPreferredName(), NAME);
|
||||
}
|
||||
builder.field(DataFrameField.INDEX_DOC_TYPE.getPreferredName(), NAME);
|
||||
builder.endObject();
|
||||
return builder;
|
||||
}
|
||||
@ -112,12 +91,14 @@ public class DataFrameTransformStateAndStats implements Writeable, ToXContentObj
|
||||
out.writeString(id);
|
||||
transformState.writeTo(out);
|
||||
transformStats.writeTo(out);
|
||||
checkpointingInfo.writeTo(out);
|
||||
if (out.getVersion().before(Version.V_7_4_0)) {
|
||||
DataFrameTransformCheckpointingInfo.EMPTY.writeTo(out);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Objects.hash(id, transformState, transformStats, checkpointingInfo);
|
||||
return Objects.hash(id, transformState, transformStats);
|
||||
}
|
||||
|
||||
@Override
|
||||
@ -130,11 +111,11 @@ public class DataFrameTransformStateAndStats implements Writeable, ToXContentObj
|
||||
return false;
|
||||
}
|
||||
|
||||
DataFrameTransformStateAndStats that = (DataFrameTransformStateAndStats) other;
|
||||
DataFrameTransformStoredDoc that = (DataFrameTransformStoredDoc) other;
|
||||
|
||||
return Objects.equals(this.id, that.id) && Objects.equals(this.transformState, that.transformState)
|
||||
&& Objects.equals(this.transformStats, that.transformStats)
|
||||
&& Objects.equals(this.checkpointingInfo, that.checkpointingInfo);
|
||||
return Objects.equals(this.id, that.id)
|
||||
&& Objects.equals(this.transformState, that.transformState)
|
||||
&& Objects.equals(this.transformStats, that.transformStats);
|
||||
}
|
||||
|
||||
public String getId() {
|
||||
@ -149,10 +130,6 @@ public class DataFrameTransformStateAndStats implements Writeable, ToXContentObj
|
||||
return transformState;
|
||||
}
|
||||
|
||||
public DataFrameTransformCheckpointingInfo getCheckpointingInfo() {
|
||||
return checkpointingInfo;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return Strings.toString(this);
|
@ -11,8 +11,8 @@ import org.elasticsearch.action.FailedNodeException;
|
||||
import org.elasticsearch.action.TaskOperationFailure;
|
||||
import org.elasticsearch.common.io.stream.Writeable.Reader;
|
||||
import org.elasticsearch.xpack.core.dataframe.action.GetDataFrameTransformsStatsAction.Response;
|
||||
import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformStateAndStats;
|
||||
import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformStateAndStatsTests;
|
||||
import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformStats;
|
||||
import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformStatsTests;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
@ -20,10 +20,10 @@ import java.util.List;
|
||||
public class GetDataFrameTransformsStatsActionResponseTests extends AbstractWireSerializingDataFrameTestCase<Response> {
|
||||
@Override
|
||||
protected Response createTestInstance() {
|
||||
List<DataFrameTransformStateAndStats> stats = new ArrayList<>();
|
||||
List<DataFrameTransformStats> stats = new ArrayList<>();
|
||||
int totalStats = randomInt(10);
|
||||
for (int i = 0; i < totalStats; ++i) {
|
||||
stats.add(DataFrameTransformStateAndStatsTests.randomDataFrameTransformStateAndStats());
|
||||
stats.add(DataFrameTransformStatsTests.randomDataFrameTransformStats());
|
||||
}
|
||||
int totalErrors = randomInt(10);
|
||||
List<TaskOperationFailure> taskFailures = new ArrayList<>(totalErrors);
|
||||
|
@ -8,13 +8,18 @@ package org.elasticsearch.xpack.core.dataframe.transforms;
|
||||
|
||||
import org.elasticsearch.common.io.stream.Writeable.Reader;
|
||||
import org.elasticsearch.common.xcontent.XContentParser;
|
||||
import org.elasticsearch.xpack.core.indexing.IndexerState;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
public class DataFrameTransformCheckpointStatsTests extends AbstractSerializingDataFrameTestCase<DataFrameTransformCheckpointStats>
|
||||
{
|
||||
public static DataFrameTransformCheckpointStats randomDataFrameTransformCheckpointStats() {
|
||||
return new DataFrameTransformCheckpointStats(randomNonNegativeLong(), randomNonNegativeLong());
|
||||
return new DataFrameTransformCheckpointStats(randomLongBetween(1, 1_000_000),
|
||||
randomBoolean() ? null : randomFrom(IndexerState.values()),
|
||||
DataFrameIndexerPositionTests.randomDataFrameIndexerPosition(),
|
||||
randomBoolean() ? null : DataFrameTransformProgressTests.randomDataFrameTransformProgress(),
|
||||
randomLongBetween(1, 1_000_000), randomLongBetween(0, 1_000_000));
|
||||
}
|
||||
|
||||
@Override
|
||||
@ -31,5 +36,4 @@ public class DataFrameTransformCheckpointStatsTests extends AbstractSerializingD
|
||||
protected Reader<DataFrameTransformCheckpointStats> instanceReader() {
|
||||
return DataFrameTransformCheckpointStats::new;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -15,6 +15,7 @@ import static org.hamcrest.Matchers.closeTo;
|
||||
import static org.hamcrest.Matchers.equalTo;
|
||||
|
||||
public class DataFrameTransformProgressTests extends AbstractSerializingDataFrameTestCase<DataFrameTransformProgress> {
|
||||
|
||||
public static DataFrameTransformProgress randomDataFrameTransformProgress() {
|
||||
long totalDocs = randomNonNegativeLong();
|
||||
return new DataFrameTransformProgress(totalDocs, randomBoolean() ? null : randomLongBetween(0, totalDocs));
|
||||
|
@ -0,0 +1,56 @@
|
||||
/*
|
||||
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||
* or more contributor license agreements. Licensed under the Elastic License;
|
||||
* you may not use this file except in compliance with the Elastic License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.xpack.core.dataframe.transforms;
|
||||
|
||||
import org.elasticsearch.common.io.stream.Writeable.Reader;
|
||||
import org.elasticsearch.common.xcontent.XContentParser;
|
||||
import org.elasticsearch.test.AbstractSerializingTestCase;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.function.Predicate;
|
||||
|
||||
public class DataFrameTransformStatsTests extends AbstractSerializingTestCase<DataFrameTransformStats> {
|
||||
|
||||
public static DataFrameTransformStats randomDataFrameTransformStats() {
|
||||
return new DataFrameTransformStats(randomAlphaOfLength(10),
|
||||
randomFrom(DataFrameTransformTaskState.values()),
|
||||
randomBoolean() ? null : randomAlphaOfLength(100),
|
||||
randomBoolean() ? null : NodeAttributeTests.randomNodeAttributes(),
|
||||
DataFrameIndexerTransformStatsTests.randomStats(DataFrameIndexerTransformStats.DEFAULT_TRANSFORM_ID),
|
||||
DataFrameTransformCheckpointingInfoTests.randomDataFrameTransformCheckpointingInfo());
|
||||
}
|
||||
|
||||
@Override
|
||||
protected DataFrameTransformStats doParseInstance(XContentParser parser) throws IOException {
|
||||
return DataFrameTransformStats.fromXContent(parser);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected DataFrameTransformStats createTestInstance() {
|
||||
return randomDataFrameTransformStats();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Reader<DataFrameTransformStats> instanceReader() {
|
||||
return DataFrameTransformStats::new;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean supportsUnknownFields() {
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected String[] getShuffleFieldsExceptions() {
|
||||
return new String[] { "position" };
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Predicate<String> getRandomFieldsExcludeFilter() {
|
||||
return field -> !field.isEmpty();
|
||||
}
|
||||
}
|
@ -14,25 +14,24 @@ import org.elasticsearch.xpack.core.dataframe.DataFrameField;
|
||||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
|
||||
public class DataFrameTransformStateAndStatsTests extends AbstractSerializingDataFrameTestCase<DataFrameTransformStateAndStats> {
|
||||
public class DataFrameTransformStoredDocTests extends AbstractSerializingDataFrameTestCase<DataFrameTransformStoredDoc> {
|
||||
|
||||
protected static ToXContent.Params TO_XCONTENT_PARAMS = new ToXContent.MapParams(
|
||||
Collections.singletonMap(DataFrameField.FOR_INTERNAL_STORAGE, "true"));
|
||||
|
||||
public static DataFrameTransformStateAndStats randomDataFrameTransformStateAndStats(String id) {
|
||||
return new DataFrameTransformStateAndStats(id,
|
||||
public static DataFrameTransformStoredDoc randomDataFrameTransformStoredDoc(String id) {
|
||||
return new DataFrameTransformStoredDoc(id,
|
||||
DataFrameTransformStateTests.randomDataFrameTransformState(),
|
||||
DataFrameIndexerTransformStatsTests.randomStats(id),
|
||||
DataFrameTransformCheckpointingInfoTests.randomDataFrameTransformCheckpointingInfo());
|
||||
DataFrameIndexerTransformStatsTests.randomStats(id));
|
||||
}
|
||||
|
||||
public static DataFrameTransformStateAndStats randomDataFrameTransformStateAndStats() {
|
||||
return randomDataFrameTransformStateAndStats(randomAlphaOfLengthBetween(1, 10));
|
||||
public static DataFrameTransformStoredDoc randomDataFrameTransformStoredDoc() {
|
||||
return randomDataFrameTransformStoredDoc(randomAlphaOfLengthBetween(1, 10));
|
||||
}
|
||||
|
||||
@Override
|
||||
protected DataFrameTransformStateAndStats doParseInstance(XContentParser parser) throws IOException {
|
||||
return DataFrameTransformStateAndStats.PARSER.apply(parser, null);
|
||||
protected DataFrameTransformStoredDoc doParseInstance(XContentParser parser) throws IOException {
|
||||
return DataFrameTransformStoredDoc.PARSER.apply(parser, null);
|
||||
}
|
||||
|
||||
@Override
|
||||
@ -43,13 +42,12 @@ public class DataFrameTransformStateAndStatsTests extends AbstractSerializingDat
|
||||
}
|
||||
|
||||
@Override
|
||||
protected DataFrameTransformStateAndStats createTestInstance() {
|
||||
return randomDataFrameTransformStateAndStats();
|
||||
protected DataFrameTransformStoredDoc createTestInstance() {
|
||||
return randomDataFrameTransformStoredDoc();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Reader<DataFrameTransformStateAndStats> instanceReader() {
|
||||
return DataFrameTransformStateAndStats::new;
|
||||
protected Reader<DataFrameTransformStoredDoc> instanceReader() {
|
||||
return DataFrameTransformStoredDoc::new;
|
||||
}
|
||||
|
||||
}
|
@ -133,9 +133,10 @@ abstract class DataFrameIntegTestCase extends ESRestTestCase {
|
||||
protected void waitUntilCheckpoint(String id, long checkpoint, TimeValue waitTime) throws Exception {
|
||||
assertBusy(() ->
|
||||
assertEquals(checkpoint, getDataFrameTransformStats(id)
|
||||
.getTransformsStateAndStats()
|
||||
.getTransformsStats()
|
||||
.get(0)
|
||||
.getTransformState()
|
||||
.getCheckpointingInfo()
|
||||
.getLast()
|
||||
.getCheckpoint()),
|
||||
waitTime.getMillis(),
|
||||
TimeUnit.MILLISECONDS);
|
||||
|
@ -65,9 +65,8 @@ public class DataFrameTransformIT extends DataFrameIntegTestCase {
|
||||
waitUntilCheckpoint(config.getId(), 1L);
|
||||
|
||||
// It will eventually be stopped
|
||||
assertBusy(() ->
|
||||
assertThat(getDataFrameTransformStats(config.getId()).getTransformsStateAndStats().get(0).getTransformState().getIndexerState(),
|
||||
equalTo(IndexerState.STOPPED)));
|
||||
assertBusy(() -> assertThat(getDataFrameTransformStats(config.getId())
|
||||
.getTransformsStats().get(0).getCheckpointingInfo().getNext().getIndexerState(), equalTo(IndexerState.STOPPED)));
|
||||
stopDataFrameTransform(config.getId());
|
||||
|
||||
DataFrameTransformConfig storedConfig = getDataFrameTransform(config.getId()).getTransformConfigurations().get(0);
|
||||
@ -103,13 +102,13 @@ public class DataFrameTransformIT extends DataFrameIntegTestCase {
|
||||
assertTrue(startDataFrameTransform(config.getId(), RequestOptions.DEFAULT).isAcknowledged());
|
||||
|
||||
waitUntilCheckpoint(config.getId(), 1L);
|
||||
assertThat(getDataFrameTransformStats(config.getId()).getTransformsStateAndStats().get(0).getTransformState().getTaskState(),
|
||||
assertThat(getDataFrameTransformStats(config.getId()).getTransformsStats().get(0).getTaskState(),
|
||||
equalTo(DataFrameTransformTaskState.STARTED));
|
||||
|
||||
long docsIndexed = getDataFrameTransformStats(config.getId())
|
||||
.getTransformsStateAndStats()
|
||||
.getTransformsStats()
|
||||
.get(0)
|
||||
.getTransformStats()
|
||||
.getIndexerStats()
|
||||
.getNumDocuments();
|
||||
|
||||
DataFrameTransformConfig storedConfig = getDataFrameTransform(config.getId()).getTransformConfigurations().get(0);
|
||||
@ -148,9 +147,9 @@ public class DataFrameTransformIT extends DataFrameIntegTestCase {
|
||||
|
||||
// Assert that we wrote the new docs
|
||||
assertThat(getDataFrameTransformStats(config.getId())
|
||||
.getTransformsStateAndStats()
|
||||
.getTransformsStats()
|
||||
.get(0)
|
||||
.getTransformStats()
|
||||
.getIndexerStats()
|
||||
.getNumDocuments(), greaterThan(docsIndexed));
|
||||
|
||||
stopDataFrameTransform(config.getId());
|
||||
|
@ -98,10 +98,13 @@ public class DataFrameGetAndGetStatsIT extends DataFrameRestTestCase {
|
||||
assertThat("documents_processed is not > 0.", ((Integer)stat.get("documents_processed")), greaterThan(0));
|
||||
assertThat("search_total is not > 0.", ((Integer)stat.get("search_total")), greaterThan(0));
|
||||
assertThat("pages_processed is not > 0.", ((Integer)stat.get("pages_processed")), greaterThan(0));
|
||||
Map<String, Object> progress = (Map<String, Object>)XContentMapValues.extractValue("state.progress", transformStats);
|
||||
/* TODO progress is now checkpoint progress and it may be that no checkpoint is in progress here
|
||||
Map<String, Object> progress =
|
||||
(Map<String, Object>)XContentMapValues.extractValue("checkpointing.next.checkpoint_progress", transformStats);
|
||||
assertThat("total_docs is not 1000", progress.get("total_docs"), equalTo(1000));
|
||||
assertThat("docs_remaining is not 0", progress.get("docs_remaining"), equalTo(0));
|
||||
assertThat("percent_complete is not 100.0", progress.get("percent_complete"), equalTo(100.0));
|
||||
*/
|
||||
}
|
||||
|
||||
// only pivot_1
|
||||
@ -111,10 +114,9 @@ public class DataFrameGetAndGetStatsIT extends DataFrameRestTestCase {
|
||||
|
||||
transformsStats = (List<Map<String, Object>>)XContentMapValues.extractValue("transforms", stats);
|
||||
assertEquals(1, transformsStats.size());
|
||||
Map<String, Object> state = (Map<String, Object>) XContentMapValues.extractValue("state", transformsStats.get(0));
|
||||
assertEquals("stopped", XContentMapValues.extractValue("task_state", state));
|
||||
assertEquals(null, XContentMapValues.extractValue("current_position", state));
|
||||
assertEquals(1, XContentMapValues.extractValue("checkpoint", state));
|
||||
assertEquals("stopped", XContentMapValues.extractValue("task_state", transformsStats.get(0)));
|
||||
assertNull(XContentMapValues.extractValue("checkpointing.next.position", transformsStats.get(0)));
|
||||
assertEquals(1, XContentMapValues.extractValue("checkpointing.last.checkpoint", transformsStats.get(0)));
|
||||
|
||||
// only continuous
|
||||
getRequest = createRequestWithAuth("GET", DATAFRAME_ENDPOINT + "pivot_continuous/_stats", authHeader);
|
||||
@ -123,9 +125,8 @@ public class DataFrameGetAndGetStatsIT extends DataFrameRestTestCase {
|
||||
|
||||
transformsStats = (List<Map<String, Object>>)XContentMapValues.extractValue("transforms", stats);
|
||||
assertEquals(1, transformsStats.size());
|
||||
state = (Map<String, Object>) XContentMapValues.extractValue("state", transformsStats.get(0));
|
||||
assertEquals("started", XContentMapValues.extractValue("task_state", state));
|
||||
assertEquals(1, XContentMapValues.extractValue("checkpoint", state));
|
||||
assertEquals("started", XContentMapValues.extractValue("task_state", transformsStats.get(0)));
|
||||
assertEquals(1, XContentMapValues.extractValue("checkpointing.last.checkpoint", transformsStats.get(0)));
|
||||
|
||||
|
||||
// check all the different ways to retrieve all transforms
|
||||
@ -198,10 +199,13 @@ public class DataFrameGetAndGetStatsIT extends DataFrameRestTestCase {
|
||||
assertThat("documents_processed is not > 0.", ((Integer)stat.get("documents_processed")), greaterThan(0));
|
||||
assertThat("search_total is not > 0.", ((Integer)stat.get("search_total")), greaterThan(0));
|
||||
assertThat("pages_processed is not > 0.", ((Integer)stat.get("pages_processed")), greaterThan(0));
|
||||
Map<String, Object> progress = (Map<String, Object>)XContentMapValues.extractValue("state.progress", transformStats);
|
||||
/* TODO progress is now checkpoint progress and it may be that no checkpoint is in progress here
|
||||
Map<String, Object> progress =
|
||||
(Map<String, Object>)XContentMapValues.extractValue("checkpointing.next.checkpoint_progress", transformStats);
|
||||
assertThat("total_docs is not 37", progress.get("total_docs"), equalTo(37));
|
||||
assertThat("docs_remaining is not 0", progress.get("docs_remaining"), equalTo(0));
|
||||
assertThat("percent_complete is not 100.0", progress.get("percent_complete"), equalTo(100.0));
|
||||
*/
|
||||
}
|
||||
}
|
||||
|
||||
@ -239,13 +243,16 @@ public class DataFrameGetAndGetStatsIT extends DataFrameRestTestCase {
|
||||
Map<String, Object> stats = entityAsMap(client().performRequest(getRequest));
|
||||
List<Map<String, Object>> transformsStats = (List<Map<String, Object>>)XContentMapValues.extractValue("transforms", stats);
|
||||
assertEquals(1, transformsStats.size());
|
||||
// Verify that the transform's progress
|
||||
/* TODO progress is now checkpoint progress and it may be that no checkpoint is in progress here
|
||||
// Verify that the transforms progress
|
||||
for (Map<String, Object> transformStats : transformsStats) {
|
||||
Map<String, Object> progress = (Map<String, Object>)XContentMapValues.extractValue("state.progress", transformStats);
|
||||
Map<String, Object> progress =
|
||||
(Map<String, Object>)XContentMapValues.extractValue("checkpointing.next.checkpoint_progress", transformStats);
|
||||
assertThat("total_docs is not 1000", progress.get("total_docs"), equalTo(1000));
|
||||
assertThat("docs_remaining is not 0", progress.get("docs_remaining"), equalTo(0));
|
||||
assertThat("percent_complete is not 100.0", progress.get("percent_complete"), equalTo(100.0));
|
||||
}
|
||||
*/
|
||||
|
||||
// add more docs to verify total_docs gets updated with continuous
|
||||
int numDocs = 10;
|
||||
@ -278,13 +285,15 @@ public class DataFrameGetAndGetStatsIT extends DataFrameRestTestCase {
|
||||
Map<String, Object> statsResponse = entityAsMap(client().performRequest(getRequest));
|
||||
List<Map<String, Object>> contStats = (List<Map<String, Object>>)XContentMapValues.extractValue("transforms", statsResponse);
|
||||
assertEquals(1, contStats.size());
|
||||
// add more docs to verify total_docs is the number of new docs added to the index
|
||||
/* TODO progress is now checkpoint progress and it may be that no checkpoint is in progress here
|
||||
for (Map<String, Object> transformStats : contStats) {
|
||||
Map<String, Object> progress = (Map<String, Object>)XContentMapValues.extractValue("state.progress", transformStats);
|
||||
Map<String, Object> progress =
|
||||
(Map<String, Object>)XContentMapValues.extractValue("checkpointing.next.checkpoint_progress", transformStats);
|
||||
assertThat("total_docs is not 10", progress.get("total_docs"), equalTo(numDocs));
|
||||
assertThat("docs_remaining is not 0", progress.get("docs_remaining"), equalTo(0));
|
||||
assertThat("percent_complete is not 100.0", progress.get("percent_complete"), equalTo(100.0));
|
||||
}
|
||||
*/
|
||||
}, 60, TimeUnit.SECONDS);
|
||||
}
|
||||
}
|
||||
|
@ -328,13 +328,17 @@ public abstract class DataFrameRestTestCase extends ESRestTestCase {
|
||||
|
||||
protected static String getDataFrameIndexerState(String transformId) throws IOException {
|
||||
Map<?, ?> transformStatsAsMap = getDataFrameState(transformId);
|
||||
return transformStatsAsMap == null ? null :
|
||||
(String) XContentMapValues.extractValue("state.indexer_state", transformStatsAsMap);
|
||||
if (transformStatsAsMap == null) {
|
||||
return null;
|
||||
}
|
||||
String indexerState = (String) XContentMapValues.extractValue("checkpointing.next.indexer_state", transformStatsAsMap);
|
||||
// If the transform is stopped then it might not have an indexer state, but logically that's the same as the indexer being stopped
|
||||
return indexerState == null ? "stopped" : indexerState;
|
||||
}
|
||||
|
||||
protected static String getDataFrameTaskState(String transformId) throws IOException {
|
||||
Map<?, ?> transformStatsAsMap = getDataFrameState(transformId);
|
||||
return transformStatsAsMap == null ? null : (String) XContentMapValues.extractValue("state.task_state", transformStatsAsMap);
|
||||
return transformStatsAsMap == null ? null : (String) XContentMapValues.extractValue("task_state", transformStatsAsMap);
|
||||
}
|
||||
|
||||
protected static Map<?, ?> getDataFrameState(String transformId) throws IOException {
|
||||
@ -423,7 +427,7 @@ public abstract class DataFrameRestTestCase extends ESRestTestCase {
|
||||
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.checkpoint", transformStatsAsMap);
|
||||
return (int) XContentMapValues.extractValue("checkpointing.last.checkpoint", transformStatsAsMap);
|
||||
}
|
||||
|
||||
protected void setupDataAccessRole(String role, String... indices) throws IOException {
|
||||
|
@ -27,7 +27,6 @@ import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
|
||||
import static org.hamcrest.CoreMatchers.is;
|
||||
import static org.hamcrest.CoreMatchers.nullValue;
|
||||
import static org.hamcrest.Matchers.equalTo;
|
||||
import static org.hamcrest.Matchers.oneOf;
|
||||
|
||||
public class DataFrameTaskFailedStateIT extends DataFrameRestTestCase {
|
||||
|
||||
@ -61,8 +60,7 @@ public class DataFrameTaskFailedStateIT extends DataFrameRestTestCase {
|
||||
final String failureReason = "task encountered more than 0 failures; latest failure: " +
|
||||
"Bulk index experienced failures. See the logs of the node running the transform for details.";
|
||||
// Verify we have failed for the expected reason
|
||||
assertThat(XContentMapValues.extractValue("state.reason", fullState),
|
||||
equalTo(failureReason));
|
||||
assertThat(XContentMapValues.extractValue("reason", fullState), equalTo(failureReason));
|
||||
|
||||
// verify that we cannot stop a failed transform
|
||||
ResponseException ex = expectThrows(ResponseException.class, () -> stopDataFrameTransform(TRANSFORM_ID, false));
|
||||
@ -77,11 +75,10 @@ public class DataFrameTaskFailedStateIT extends DataFrameRestTestCase {
|
||||
|
||||
awaitState(TRANSFORM_ID, DataFrameTransformTaskState.STOPPED);
|
||||
fullState = getDataFrameState(TRANSFORM_ID);
|
||||
assertThat(XContentMapValues.extractValue("state.reason", fullState),
|
||||
is(nullValue()));
|
||||
// Verify we have failed for the expected reason
|
||||
assertThat(XContentMapValues.extractValue("reason", fullState), is(nullValue()));
|
||||
}
|
||||
|
||||
|
||||
public void testForceStartFailedTransform() throws Exception {
|
||||
createReviewsIndex(REVIEWS_INDEX_NAME, 10);
|
||||
String dataFrameIndex = "failure_pivot_reviews";
|
||||
@ -93,8 +90,7 @@ public class DataFrameTaskFailedStateIT extends DataFrameRestTestCase {
|
||||
final String failureReason = "task encountered more than 0 failures; latest failure: " +
|
||||
"Bulk index experienced failures. See the logs of the node running the transform for details.";
|
||||
// Verify we have failed for the expected reason
|
||||
assertThat(XContentMapValues.extractValue("state.reason", fullState),
|
||||
equalTo(failureReason));
|
||||
assertThat(XContentMapValues.extractValue("reason", fullState), equalTo(failureReason));
|
||||
|
||||
// Verify that we cannot start the transform when the task is in a failed state
|
||||
ResponseException ex = expectThrows(ResponseException.class, () -> startDataframeTransform(TRANSFORM_ID, false));
|
||||
@ -111,10 +107,9 @@ public class DataFrameTaskFailedStateIT extends DataFrameRestTestCase {
|
||||
|
||||
// Verify that we have started and that our reason is cleared
|
||||
fullState = getDataFrameState(TRANSFORM_ID);
|
||||
assertThat(XContentMapValues.extractValue("state.reason", fullState), is(nullValue()));
|
||||
assertThat(XContentMapValues.extractValue("state.task_state", fullState), equalTo("started"));
|
||||
assertThat(XContentMapValues.extractValue("state.indexer_state", fullState), is(oneOf("started", "indexing")));
|
||||
assertThat(XContentMapValues.extractValue("stats.index_failures", fullState), equalTo(1));
|
||||
assertThat(XContentMapValues.extractValue("reason", fullState), is(nullValue()));
|
||||
assertThat(XContentMapValues.extractValue("task_state", fullState), equalTo("started"));
|
||||
assertThat((int)XContentMapValues.extractValue("stats.index_failures", fullState), equalTo(1));
|
||||
|
||||
stopDataFrameTransform(TRANSFORM_ID, true);
|
||||
}
|
||||
|
@ -11,7 +11,7 @@ import org.elasticsearch.client.Response;
|
||||
import org.elasticsearch.common.xcontent.support.XContentMapValues;
|
||||
|
||||
import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameIndexerTransformStats;
|
||||
import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformStateAndStats;
|
||||
import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformStoredDoc;
|
||||
import org.elasticsearch.xpack.dataframe.persistence.DataFrameInternalIndex;
|
||||
import org.junit.Before;
|
||||
|
||||
@ -56,7 +56,7 @@ public class DataFrameUsageIT extends DataFrameRestTestCase {
|
||||
Request statsExistsRequest = new Request("GET",
|
||||
DataFrameInternalIndex.INDEX_NAME+"/_search?q=" +
|
||||
INDEX_DOC_TYPE.getPreferredName() + ":" +
|
||||
DataFrameTransformStateAndStats.NAME);
|
||||
DataFrameTransformStoredDoc.NAME);
|
||||
// Verify that we have one stat document
|
||||
assertBusy(() -> {
|
||||
Map<String, Object> hasStatsMap = entityAsMap(client().performRequest(statsExistsRequest));
|
||||
|
@ -35,7 +35,7 @@ import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameIndexerTransfo
|
||||
import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransform;
|
||||
import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformConfig;
|
||||
import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformState;
|
||||
import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformStateAndStats;
|
||||
import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformStoredDoc;
|
||||
import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformTaskState;
|
||||
import org.elasticsearch.xpack.dataframe.persistence.DataFrameInternalIndex;
|
||||
|
||||
@ -194,7 +194,7 @@ public class DataFrameFeatureSet implements XPackFeatureSet {
|
||||
static void getStatisticSummations(Client client, ActionListener<DataFrameIndexerTransformStats> statsListener) {
|
||||
QueryBuilder queryBuilder = QueryBuilders.constantScoreQuery(QueryBuilders.boolQuery()
|
||||
.filter(QueryBuilders.termQuery(DataFrameField.INDEX_DOC_TYPE.getPreferredName(),
|
||||
DataFrameTransformStateAndStats.NAME)));
|
||||
DataFrameTransformStoredDoc.NAME)));
|
||||
|
||||
SearchRequestBuilder requestBuilder = client.prepareSearch(DataFrameInternalIndex.INDEX_NAME)
|
||||
.setSize(0)
|
||||
|
@ -27,11 +27,10 @@ import org.elasticsearch.xpack.core.dataframe.action.GetDataFrameTransformsStats
|
||||
import org.elasticsearch.xpack.core.dataframe.action.GetDataFrameTransformsStatsAction.Request;
|
||||
import org.elasticsearch.xpack.core.dataframe.action.GetDataFrameTransformsStatsAction.Response;
|
||||
import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformCheckpointingInfo;
|
||||
import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformState;
|
||||
import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformStateAndStats;
|
||||
import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformStats;
|
||||
import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformStoredDoc;
|
||||
import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformTaskState;
|
||||
import org.elasticsearch.xpack.core.dataframe.transforms.NodeAttributes;
|
||||
import org.elasticsearch.xpack.core.indexing.IndexerState;
|
||||
import org.elasticsearch.xpack.dataframe.checkpoint.DataFrameTransformsCheckpointService;
|
||||
import org.elasticsearch.xpack.dataframe.persistence.DataFrameTransformsConfigManager;
|
||||
import org.elasticsearch.xpack.dataframe.transforms.DataFrameTransformTask;
|
||||
@ -42,6 +41,8 @@ import java.util.Comparator;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
public class TransportGetDataFrameTransformsStatsAction extends
|
||||
@ -69,9 +70,9 @@ public class TransportGetDataFrameTransformsStatsAction extends
|
||||
@Override
|
||||
protected Response newResponse(Request request, List<Response> tasks, List<TaskOperationFailure> taskOperationFailures,
|
||||
List<FailedNodeException> failedNodeExceptions) {
|
||||
List<DataFrameTransformStateAndStats> responses = tasks.stream()
|
||||
.flatMap(r -> r.getTransformsStateAndStats().stream())
|
||||
.sorted(Comparator.comparing(DataFrameTransformStateAndStats::getId))
|
||||
List<DataFrameTransformStats> responses = tasks.stream()
|
||||
.flatMap(r -> r.getTransformsStats().stream())
|
||||
.sorted(Comparator.comparing(DataFrameTransformStats::getId))
|
||||
.collect(Collectors.toList());
|
||||
List<ElasticsearchException> allFailedNodeExceptions = new ArrayList<>(failedNodeExceptions);
|
||||
allFailedNodeExceptions.addAll(tasks.stream().flatMap(r -> r.getNodeFailures().stream()).collect(Collectors.toList()));
|
||||
@ -85,15 +86,20 @@ public class TransportGetDataFrameTransformsStatsAction extends
|
||||
String nodeId = state.nodes().getLocalNode().getId();
|
||||
if (task.isCancelled() == false) {
|
||||
transformsCheckpointService.getCheckpointStats(task.getTransformId(), task.getCheckpoint(), task.getInProgressCheckpoint(),
|
||||
task.getState().getIndexerState(), task.getState().getPosition(), task.getProgress(),
|
||||
ActionListener.wrap(checkpointStats -> listener.onResponse(new Response(
|
||||
Collections.singletonList(new DataFrameTransformStateAndStats(task.getTransformId(),
|
||||
task.getState(),
|
||||
Collections.singletonList(new DataFrameTransformStats(task.getTransformId(),
|
||||
task.getState().getTaskState(),
|
||||
task.getState().getReason(),
|
||||
null,
|
||||
task.getStats(),
|
||||
checkpointStats)),
|
||||
1L)),
|
||||
e -> listener.onResponse(new Response(
|
||||
Collections.singletonList(new DataFrameTransformStateAndStats(task.getTransformId(),
|
||||
task.getState(),
|
||||
Collections.singletonList(new DataFrameTransformStats(task.getTransformId(),
|
||||
task.getState().getTaskState(),
|
||||
task.getState().getReason(),
|
||||
null,
|
||||
task.getStats(),
|
||||
DataFrameTransformCheckpointingInfo.EMPTY)),
|
||||
1L,
|
||||
@ -119,10 +125,10 @@ public class TransportGetDataFrameTransformsStatsAction extends
|
||||
PersistentTasksCustomMetaData tasksInProgress = state.getMetaData().custom(PersistentTasksCustomMetaData.TYPE);
|
||||
if (tasksInProgress != null) {
|
||||
// Mutates underlying state object with the assigned node attributes
|
||||
response.getTransformsStateAndStats().forEach(dtsas -> setNodeAttributes(dtsas, tasksInProgress, state));
|
||||
response.getTransformsStats().forEach(dtsasi -> setNodeAttributes(dtsasi, tasksInProgress, state));
|
||||
}
|
||||
collectStatsForTransformsWithoutTasks(request, response, ActionListener.wrap(
|
||||
finalResponse -> finalListener.onResponse(new Response(finalResponse.getTransformsStateAndStats(),
|
||||
finalResponse -> finalListener.onResponse(new Response(finalResponse.getTransformsStats(),
|
||||
hitsAndIds.v1(),
|
||||
finalResponse.getTaskFailures(),
|
||||
finalResponse.getNodeFailures())),
|
||||
@ -143,14 +149,13 @@ public class TransportGetDataFrameTransformsStatsAction extends
|
||||
));
|
||||
}
|
||||
|
||||
private static void setNodeAttributes(DataFrameTransformStateAndStats dataFrameTransformStateAndStats,
|
||||
PersistentTasksCustomMetaData persistentTasksCustomMetaData,
|
||||
ClusterState state) {
|
||||
private static void setNodeAttributes(DataFrameTransformStats dataFrameTransformStats,
|
||||
PersistentTasksCustomMetaData persistentTasksCustomMetaData,
|
||||
ClusterState state) {
|
||||
PersistentTasksCustomMetaData.PersistentTask<?> pTask =
|
||||
persistentTasksCustomMetaData.getTask(dataFrameTransformStateAndStats.getTransformId());
|
||||
persistentTasksCustomMetaData.getTask(dataFrameTransformStats.getId());
|
||||
if (pTask != null) {
|
||||
dataFrameTransformStateAndStats.getTransformState()
|
||||
.setNode(NodeAttributes.fromDiscoveryNode(state.nodes().get(pTask.getExecutorNode())));
|
||||
dataFrameTransformStats.setNode(NodeAttributes.fromDiscoveryNode(state.nodes().get(pTask.getExecutorNode())));
|
||||
}
|
||||
}
|
||||
|
||||
@ -158,51 +163,44 @@ public class TransportGetDataFrameTransformsStatsAction extends
|
||||
Response response,
|
||||
ActionListener<Response> listener) {
|
||||
// We gathered all there is, no need to continue
|
||||
if (request.getExpandedIds().size() == response.getTransformsStateAndStats().size()) {
|
||||
if (request.getExpandedIds().size() == response.getTransformsStats().size()) {
|
||||
listener.onResponse(response);
|
||||
return;
|
||||
}
|
||||
|
||||
Set<String> transformsWithoutTasks = new HashSet<>(request.getExpandedIds());
|
||||
transformsWithoutTasks.removeAll(response.getTransformsStateAndStats().stream().map(DataFrameTransformStateAndStats::getId)
|
||||
transformsWithoutTasks.removeAll(response.getTransformsStats().stream().map(DataFrameTransformStats::getId)
|
||||
.collect(Collectors.toList()));
|
||||
|
||||
// Small assurance that we are at least below the max. Terms search has a hard limit of 10k, we should at least be below that.
|
||||
assert transformsWithoutTasks.size() <= Request.MAX_SIZE_RETURN;
|
||||
|
||||
ActionListener<List<DataFrameTransformStateAndStats>> searchStatsListener = ActionListener.wrap(
|
||||
stats -> {
|
||||
List<DataFrameTransformStateAndStats> allStateAndStats = response.getTransformsStateAndStats();
|
||||
// If the persistent task does NOT exist, it is STOPPED
|
||||
// There is a potential race condition where the saved document does not actually have a STOPPED state
|
||||
// as the task is cancelled before we persist state.
|
||||
stats.forEach(stat ->
|
||||
allStateAndStats.add(new DataFrameTransformStateAndStats(
|
||||
stat.getId(),
|
||||
new DataFrameTransformState(DataFrameTransformTaskState.STOPPED,
|
||||
IndexerState.STOPPED,
|
||||
stat.getTransformState().getPosition(),
|
||||
stat.getTransformState().getCheckpoint(),
|
||||
stat.getTransformState().getReason(),
|
||||
stat.getTransformState().getProgress()),
|
||||
stat.getTransformStats(),
|
||||
stat.getCheckpointingInfo()))
|
||||
);
|
||||
transformsWithoutTasks.removeAll(
|
||||
stats.stream().map(DataFrameTransformStateAndStats::getId).collect(Collectors.toSet()));
|
||||
// If the persistent task does NOT exist, it is STOPPED
|
||||
// There is a potential race condition where the saved document does not actually have a STOPPED state
|
||||
// as the task is cancelled before we persist state.
|
||||
ActionListener<List<DataFrameTransformStoredDoc>> searchStatsListener = ActionListener.wrap(
|
||||
statsForTransformsWithoutTasks -> {
|
||||
List<DataFrameTransformStats> allStateAndStats = response.getTransformsStats();
|
||||
addCheckpointingInfoForTransformsWithoutTasks(allStateAndStats, statsForTransformsWithoutTasks,
|
||||
ActionListener.wrap(
|
||||
aVoid -> {
|
||||
transformsWithoutTasks.removeAll(statsForTransformsWithoutTasks.stream()
|
||||
.map(DataFrameTransformStoredDoc::getId).collect(Collectors.toSet()));
|
||||
|
||||
// Transforms that have not been started and have no state or stats.
|
||||
transformsWithoutTasks.forEach(transformId ->
|
||||
allStateAndStats.add(DataFrameTransformStateAndStats.initialStateAndStats(transformId)));
|
||||
// Transforms that have not been started and have no state or stats.
|
||||
transformsWithoutTasks.forEach(
|
||||
transformId -> allStateAndStats.add(DataFrameTransformStats.initialStats(transformId)));
|
||||
|
||||
// Any transform in collection could NOT have a task, so, even though the list is initially sorted
|
||||
// it can easily become arbitrarily ordered based on which transforms don't have a task or stats docs
|
||||
allStateAndStats.sort(Comparator.comparing(DataFrameTransformStateAndStats::getId));
|
||||
// Any transform in collection could NOT have a task, so, even though the list is initially sorted
|
||||
// it can easily become arbitrarily ordered based on which transforms don't have a task or stats docs
|
||||
allStateAndStats.sort(Comparator.comparing(DataFrameTransformStats::getId));
|
||||
|
||||
listener.onResponse(new Response(allStateAndStats,
|
||||
allStateAndStats.size(),
|
||||
response.getTaskFailures(),
|
||||
response.getNodeFailures()));
|
||||
listener.onResponse(new Response(allStateAndStats,
|
||||
allStateAndStats.size(),
|
||||
response.getTaskFailures(),
|
||||
response.getNodeFailures()));
|
||||
},
|
||||
listener::onFailure));
|
||||
},
|
||||
e -> {
|
||||
if (e instanceof IndexNotFoundException) {
|
||||
@ -213,6 +211,57 @@ public class TransportGetDataFrameTransformsStatsAction extends
|
||||
}
|
||||
);
|
||||
|
||||
dataFrameTransformsConfigManager.getTransformStats(transformsWithoutTasks, searchStatsListener);
|
||||
dataFrameTransformsConfigManager.getTransformStoredDoc(transformsWithoutTasks, searchStatsListener);
|
||||
}
|
||||
|
||||
private void populateSingleStoppedTransformStat(DataFrameTransformStoredDoc transform,
|
||||
ActionListener<DataFrameTransformCheckpointingInfo> listener) {
|
||||
transformsCheckpointService.getCheckpointStats(transform.getId(), transform.getTransformState().getCheckpoint(),
|
||||
transform.getTransformState().getCheckpoint() + 1, transform.getTransformState().getIndexerState(),
|
||||
transform.getTransformState().getPosition(), transform.getTransformState().getProgress(),
|
||||
ActionListener.wrap(
|
||||
listener::onResponse,
|
||||
e -> {
|
||||
logger.warn("Failed to retrieve checkpointing info for transform [" + transform.getId() + "]", e);
|
||||
listener.onResponse(DataFrameTransformCheckpointingInfo.EMPTY);
|
||||
}
|
||||
));
|
||||
}
|
||||
|
||||
private void addCheckpointingInfoForTransformsWithoutTasks(List<DataFrameTransformStats> allStateAndStats,
|
||||
List<DataFrameTransformStoredDoc> statsForTransformsWithoutTasks,
|
||||
ActionListener<Void> listener) {
|
||||
|
||||
if (statsForTransformsWithoutTasks.isEmpty()) {
|
||||
// No work to do, but we must respond to the listener
|
||||
listener.onResponse(null);
|
||||
return;
|
||||
}
|
||||
|
||||
AtomicInteger numberRemaining = new AtomicInteger(statsForTransformsWithoutTasks.size());
|
||||
AtomicBoolean isExceptionReported = new AtomicBoolean(false);
|
||||
|
||||
statsForTransformsWithoutTasks.forEach(stat -> populateSingleStoppedTransformStat(stat,
|
||||
ActionListener.wrap(
|
||||
checkpointingInfo -> {
|
||||
synchronized (allStateAndStats) {
|
||||
allStateAndStats.add(new DataFrameTransformStats(
|
||||
stat.getId(),
|
||||
DataFrameTransformTaskState.STOPPED,
|
||||
null,
|
||||
null,
|
||||
stat.getTransformStats(),
|
||||
checkpointingInfo));
|
||||
}
|
||||
if (numberRemaining.decrementAndGet() == 0) {
|
||||
listener.onResponse(null);
|
||||
}
|
||||
},
|
||||
e -> {
|
||||
if (isExceptionReported.compareAndSet(false, true)) {
|
||||
listener.onFailure(e);
|
||||
}
|
||||
}
|
||||
)));
|
||||
}
|
||||
}
|
||||
|
@ -17,12 +17,15 @@ import org.elasticsearch.action.admin.indices.stats.ShardStats;
|
||||
import org.elasticsearch.action.support.IndicesOptions;
|
||||
import org.elasticsearch.client.Client;
|
||||
import org.elasticsearch.xpack.core.ClientHelper;
|
||||
import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameIndexerPosition;
|
||||
import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformCheckpoint;
|
||||
import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformCheckpointStats;
|
||||
import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformCheckpointingInfo;
|
||||
import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformConfig;
|
||||
import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformProgress;
|
||||
import org.elasticsearch.xpack.core.dataframe.transforms.SyncConfig;
|
||||
import org.elasticsearch.xpack.core.dataframe.transforms.TimeSyncConfig;
|
||||
import org.elasticsearch.xpack.core.indexing.IndexerState;
|
||||
import org.elasticsearch.xpack.dataframe.persistence.DataFrameTransformsConfigManager;
|
||||
|
||||
import java.util.Arrays;
|
||||
@ -42,17 +45,32 @@ import java.util.TreeMap;
|
||||
public class DataFrameTransformsCheckpointService {
|
||||
|
||||
private static class Checkpoints {
|
||||
DataFrameTransformCheckpoint currentCheckpoint = DataFrameTransformCheckpoint.EMPTY;
|
||||
DataFrameTransformCheckpoint inProgressCheckpoint = DataFrameTransformCheckpoint.EMPTY;
|
||||
long lastCheckpointNumber;
|
||||
long nextCheckpointNumber;
|
||||
IndexerState nextCheckpointIndexerState;
|
||||
DataFrameIndexerPosition nextCheckpointPosition;
|
||||
DataFrameTransformProgress nextCheckpointProgress;
|
||||
DataFrameTransformCheckpoint lastCheckpoint = DataFrameTransformCheckpoint.EMPTY;
|
||||
DataFrameTransformCheckpoint nextCheckpoint = DataFrameTransformCheckpoint.EMPTY;
|
||||
DataFrameTransformCheckpoint sourceCheckpoint = DataFrameTransformCheckpoint.EMPTY;
|
||||
|
||||
Checkpoints(long lastCheckpointNumber, long nextCheckpointNumber, IndexerState nextCheckpointIndexerState,
|
||||
DataFrameIndexerPosition nextCheckpointPosition, DataFrameTransformProgress nextCheckpointProgress) {
|
||||
this.lastCheckpointNumber = lastCheckpointNumber;
|
||||
this.nextCheckpointNumber = nextCheckpointNumber;
|
||||
this.nextCheckpointIndexerState = nextCheckpointIndexerState;
|
||||
this.nextCheckpointPosition = nextCheckpointPosition;
|
||||
this.nextCheckpointProgress = nextCheckpointProgress;
|
||||
}
|
||||
|
||||
DataFrameTransformCheckpointingInfo buildInfo() {
|
||||
return new DataFrameTransformCheckpointingInfo(
|
||||
new DataFrameTransformCheckpointStats(currentCheckpoint.getTimestamp(), currentCheckpoint.getTimeUpperBound()),
|
||||
new DataFrameTransformCheckpointStats(inProgressCheckpoint.getTimestamp(), inProgressCheckpoint.getTimeUpperBound()),
|
||||
DataFrameTransformCheckpoint.getBehind(currentCheckpoint, sourceCheckpoint));
|
||||
new DataFrameTransformCheckpointStats(lastCheckpointNumber, null, null, null,
|
||||
lastCheckpoint.getTimestamp(), lastCheckpoint.getTimeUpperBound()),
|
||||
new DataFrameTransformCheckpointStats(nextCheckpointNumber, nextCheckpointIndexerState, nextCheckpointPosition,
|
||||
nextCheckpointProgress, nextCheckpoint.getTimestamp(), nextCheckpoint.getTimeUpperBound()),
|
||||
DataFrameTransformCheckpoint.getBehind(lastCheckpoint, sourceCheckpoint));
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
private static final Logger logger = LogManager.getLogger(DataFrameTransformsCheckpointService.class);
|
||||
@ -84,7 +102,7 @@ public class DataFrameTransformsCheckpointService {
|
||||
* @param listener listener to call after inner request returned
|
||||
*/
|
||||
public void getCheckpoint(DataFrameTransformConfig transformConfig, long checkpoint,
|
||||
ActionListener<DataFrameTransformCheckpoint> listener) {
|
||||
ActionListener<DataFrameTransformCheckpoint> listener) {
|
||||
long timestamp = System.currentTimeMillis();
|
||||
|
||||
// for time based synchronization
|
||||
@ -127,55 +145,59 @@ public class DataFrameTransformsCheckpointService {
|
||||
},
|
||||
e -> listener.onFailure(new CheckpointException("Failed to create checkpoint", e))
|
||||
));
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* Get checkpointing stats for a data frame
|
||||
*
|
||||
*
|
||||
* @param transformId The data frame task
|
||||
* @param currentCheckpoint the current checkpoint
|
||||
* @param inProgressCheckpoint in progress checkpoint
|
||||
* @param lastCheckpoint the last checkpoint
|
||||
* @param nextCheckpoint the next checkpoint
|
||||
* @param nextCheckpointIndexerState indexer state for the next checkpoint
|
||||
* @param nextCheckpointPosition position for the next checkpoint
|
||||
* @param nextCheckpointProgress progress for the next checkpoint
|
||||
* @param listener listener to retrieve the result
|
||||
*/
|
||||
public void getCheckpointStats(
|
||||
String transformId,
|
||||
long currentCheckpoint,
|
||||
long inProgressCheckpoint,
|
||||
ActionListener<DataFrameTransformCheckpointingInfo> listener) {
|
||||
public void getCheckpointStats(String transformId,
|
||||
long lastCheckpoint,
|
||||
long nextCheckpoint,
|
||||
IndexerState nextCheckpointIndexerState,
|
||||
DataFrameIndexerPosition nextCheckpointPosition,
|
||||
DataFrameTransformProgress nextCheckpointProgress,
|
||||
ActionListener<DataFrameTransformCheckpointingInfo> listener) {
|
||||
|
||||
Checkpoints checkpoints = new Checkpoints();
|
||||
Checkpoints checkpoints =
|
||||
new Checkpoints(lastCheckpoint, nextCheckpoint, nextCheckpointIndexerState, nextCheckpointPosition, nextCheckpointProgress);
|
||||
|
||||
// <3> notify the user once we have the current checkpoint
|
||||
ActionListener<DataFrameTransformCheckpoint> currentCheckpointListener = ActionListener.wrap(
|
||||
currentCheckpointObj -> {
|
||||
checkpoints.currentCheckpoint = currentCheckpointObj;
|
||||
// <3> notify the user once we have the last checkpoint
|
||||
ActionListener<DataFrameTransformCheckpoint> lastCheckpointListener = ActionListener.wrap(
|
||||
lastCheckpointObj -> {
|
||||
checkpoints.lastCheckpoint = lastCheckpointObj;
|
||||
listener.onResponse(checkpoints.buildInfo());
|
||||
},
|
||||
e -> {
|
||||
logger.debug("Failed to retrieve current checkpoint [" +
|
||||
currentCheckpoint + "] for data frame [" + transformId + "]", e);
|
||||
listener.onFailure(new CheckpointException("Failure during current checkpoint info retrieval", e));
|
||||
logger.debug("Failed to retrieve last checkpoint [" +
|
||||
lastCheckpoint + "] for data frame [" + transformId + "]", e);
|
||||
listener.onFailure(new CheckpointException("Failure during last checkpoint info retrieval", e));
|
||||
}
|
||||
);
|
||||
|
||||
// <2> after the in progress checkpoint, get the current checkpoint
|
||||
ActionListener<DataFrameTransformCheckpoint> inProgressCheckpointListener = ActionListener.wrap(
|
||||
inProgressCheckpointObj -> {
|
||||
checkpoints.inProgressCheckpoint = inProgressCheckpointObj;
|
||||
if (currentCheckpoint != 0) {
|
||||
// <2> after the next checkpoint, get the last checkpoint
|
||||
ActionListener<DataFrameTransformCheckpoint> nextCheckpointListener = ActionListener.wrap(
|
||||
nextCheckpointObj -> {
|
||||
checkpoints.nextCheckpoint = nextCheckpointObj;
|
||||
if (lastCheckpoint != 0) {
|
||||
dataFrameTransformsConfigManager.getTransformCheckpoint(transformId,
|
||||
currentCheckpoint,
|
||||
currentCheckpointListener);
|
||||
lastCheckpoint,
|
||||
lastCheckpointListener);
|
||||
} else {
|
||||
currentCheckpointListener.onResponse(DataFrameTransformCheckpoint.EMPTY);
|
||||
lastCheckpointListener.onResponse(DataFrameTransformCheckpoint.EMPTY);
|
||||
}
|
||||
},
|
||||
e -> {
|
||||
logger.debug("Failed to retrieve in progress checkpoint [" +
|
||||
inProgressCheckpoint + "] for data frame [" + transformId + "]", e);
|
||||
listener.onFailure(new CheckpointException("Failure during in progress checkpoint info retrieval", e));
|
||||
logger.debug("Failed to retrieve next checkpoint [" +
|
||||
nextCheckpoint + "] for data frame [" + transformId + "]", e);
|
||||
listener.onFailure(new CheckpointException("Failure during next checkpoint info retrieval", e));
|
||||
}
|
||||
);
|
||||
|
||||
@ -183,12 +205,12 @@ public class DataFrameTransformsCheckpointService {
|
||||
ActionListener<DataFrameTransformCheckpoint> sourceCheckpointListener = ActionListener.wrap(
|
||||
sourceCheckpoint -> {
|
||||
checkpoints.sourceCheckpoint = sourceCheckpoint;
|
||||
if (inProgressCheckpoint != 0) {
|
||||
if (nextCheckpoint != 0) {
|
||||
dataFrameTransformsConfigManager.getTransformCheckpoint(transformId,
|
||||
inProgressCheckpoint,
|
||||
inProgressCheckpointListener);
|
||||
nextCheckpoint,
|
||||
nextCheckpointListener);
|
||||
} else {
|
||||
inProgressCheckpointListener.onResponse(DataFrameTransformCheckpoint.EMPTY);
|
||||
nextCheckpointListener.onResponse(DataFrameTransformCheckpoint.EMPTY);
|
||||
}
|
||||
},
|
||||
e -> {
|
||||
|
@ -19,7 +19,7 @@ import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameIndexerTransfo
|
||||
import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformConfig;
|
||||
import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformProgress;
|
||||
import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformState;
|
||||
import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformStateAndStats;
|
||||
import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformStoredDoc;
|
||||
import org.elasticsearch.xpack.core.dataframe.transforms.DestConfig;
|
||||
import org.elasticsearch.xpack.core.dataframe.transforms.SourceConfig;
|
||||
|
||||
@ -133,7 +133,7 @@ public final class DataFrameInternalIndex {
|
||||
// add the schema for transform configurations
|
||||
addDataFrameTransformsConfigMappings(builder);
|
||||
// add the schema for transform stats
|
||||
addDataFrameTransformStateAndStatsMappings(builder);
|
||||
addDataFrameTransformStoredDocMappings(builder);
|
||||
// end type
|
||||
builder.endObject();
|
||||
// end properties
|
||||
@ -144,9 +144,9 @@ public final class DataFrameInternalIndex {
|
||||
}
|
||||
|
||||
|
||||
private static XContentBuilder addDataFrameTransformStateAndStatsMappings(XContentBuilder builder) throws IOException {
|
||||
private static XContentBuilder addDataFrameTransformStoredDocMappings(XContentBuilder builder) throws IOException {
|
||||
return builder
|
||||
.startObject(DataFrameTransformStateAndStats.STATE_FIELD.getPreferredName())
|
||||
.startObject(DataFrameTransformStoredDoc.STATE_FIELD.getPreferredName())
|
||||
.startObject(PROPERTIES)
|
||||
.startObject(DataFrameTransformState.TASK_STATE.getPreferredName())
|
||||
.field(TYPE, KEYWORD)
|
||||
@ -212,7 +212,9 @@ public final class DataFrameInternalIndex {
|
||||
.endObject()
|
||||
.endObject()
|
||||
.endObject()
|
||||
.startObject(DataFrameTransformStateAndStats.CHECKPOINTING_INFO_FIELD.getPreferredName())
|
||||
// This is obsolete and can be removed for future versions of the index, but is left here as a warning/reminder that
|
||||
// we cannot declare this field differently in version 1 of the internal index as it would cause a mapping clash
|
||||
.startObject("checkpointing")
|
||||
.field(ENABLED, false)
|
||||
.endObject();
|
||||
}
|
||||
|
@ -47,7 +47,7 @@ import org.elasticsearch.xpack.core.dataframe.DataFrameField;
|
||||
import org.elasticsearch.xpack.core.dataframe.DataFrameMessages;
|
||||
import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformCheckpoint;
|
||||
import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformConfig;
|
||||
import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformStateAndStats;
|
||||
import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformStoredDoc;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
@ -280,31 +280,31 @@ public class DataFrameTransformsConfigManager {
|
||||
}));
|
||||
}
|
||||
|
||||
public void putOrUpdateTransformStats(DataFrameTransformStateAndStats stats, ActionListener<Boolean> listener) {
|
||||
public void putOrUpdateTransformStoredDoc(DataFrameTransformStoredDoc stats, ActionListener<Boolean> listener) {
|
||||
try (XContentBuilder builder = XContentFactory.jsonBuilder()) {
|
||||
XContentBuilder source = stats.toXContent(builder, new ToXContent.MapParams(TO_XCONTENT_PARAMS));
|
||||
|
||||
IndexRequest indexRequest = new IndexRequest(DataFrameInternalIndex.INDEX_NAME)
|
||||
.setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE)
|
||||
.id(DataFrameTransformStateAndStats.documentId(stats.getTransformId()))
|
||||
.id(DataFrameTransformStoredDoc.documentId(stats.getId()))
|
||||
.source(source);
|
||||
|
||||
executeAsyncWithOrigin(client, DATA_FRAME_ORIGIN, IndexAction.INSTANCE, indexRequest, ActionListener.wrap(
|
||||
r -> listener.onResponse(true),
|
||||
e -> listener.onFailure(new RuntimeException(
|
||||
DataFrameMessages.getMessage(DataFrameMessages.DATA_FRAME_FAILED_TO_PERSIST_STATS, stats.getTransformId()),
|
||||
DataFrameMessages.getMessage(DataFrameMessages.DATA_FRAME_FAILED_TO_PERSIST_STATS, stats.getId()),
|
||||
e))
|
||||
));
|
||||
} catch (IOException e) {
|
||||
// not expected to happen but for the sake of completeness
|
||||
listener.onFailure(new ElasticsearchParseException(
|
||||
DataFrameMessages.getMessage(DataFrameMessages.DATA_FRAME_FAILED_TO_PERSIST_STATS, stats.getTransformId()),
|
||||
DataFrameMessages.getMessage(DataFrameMessages.DATA_FRAME_FAILED_TO_PERSIST_STATS, stats.getId()),
|
||||
e));
|
||||
}
|
||||
}
|
||||
|
||||
public void getTransformStats(String transformId, ActionListener<DataFrameTransformStateAndStats> resultListener) {
|
||||
GetRequest getRequest = new GetRequest(DataFrameInternalIndex.INDEX_NAME, DataFrameTransformStateAndStats.documentId(transformId));
|
||||
public void getTransformStoredDoc(String transformId, ActionListener<DataFrameTransformStoredDoc> resultListener) {
|
||||
GetRequest getRequest = new GetRequest(DataFrameInternalIndex.INDEX_NAME, DataFrameTransformStoredDoc.documentId(transformId));
|
||||
executeAsyncWithOrigin(client, DATA_FRAME_ORIGIN, GetAction.INSTANCE, getRequest, ActionListener.wrap(getResponse -> {
|
||||
|
||||
if (getResponse.isExists() == false) {
|
||||
@ -316,7 +316,7 @@ public class DataFrameTransformsConfigManager {
|
||||
try (InputStream stream = source.streamInput();
|
||||
XContentParser parser = XContentFactory.xContent(XContentType.JSON)
|
||||
.createParser(xContentRegistry, LoggingDeprecationHandler.INSTANCE, stream)) {
|
||||
resultListener.onResponse(DataFrameTransformStateAndStats.fromXContent(parser));
|
||||
resultListener.onResponse(DataFrameTransformStoredDoc.fromXContent(parser));
|
||||
} catch (Exception e) {
|
||||
logger.error(
|
||||
DataFrameMessages.getMessage(DataFrameMessages.FAILED_TO_PARSE_TRANSFORM_STATISTICS_CONFIGURATION, transformId), e);
|
||||
@ -332,11 +332,11 @@ public class DataFrameTransformsConfigManager {
|
||||
}));
|
||||
}
|
||||
|
||||
public void getTransformStats(Collection<String> transformIds, ActionListener<List<DataFrameTransformStateAndStats>> listener) {
|
||||
public void getTransformStoredDoc(Collection<String> transformIds, ActionListener<List<DataFrameTransformStoredDoc>> listener) {
|
||||
|
||||
QueryBuilder builder = QueryBuilders.constantScoreQuery(QueryBuilders.boolQuery()
|
||||
.filter(QueryBuilders.termsQuery(DataFrameField.ID.getPreferredName(), transformIds))
|
||||
.filter(QueryBuilders.termQuery(DataFrameField.INDEX_DOC_TYPE.getPreferredName(), DataFrameTransformStateAndStats.NAME)));
|
||||
.filter(QueryBuilders.termQuery(DataFrameField.INDEX_DOC_TYPE.getPreferredName(), DataFrameTransformStoredDoc.NAME)));
|
||||
|
||||
SearchRequest searchRequest = client.prepareSearch(DataFrameInternalIndex.INDEX_NAME)
|
||||
.addSort(DataFrameField.ID.getPreferredName(), SortOrder.ASC)
|
||||
@ -347,13 +347,13 @@ public class DataFrameTransformsConfigManager {
|
||||
executeAsyncWithOrigin(client.threadPool().getThreadContext(), DATA_FRAME_ORIGIN, searchRequest,
|
||||
ActionListener.<SearchResponse>wrap(
|
||||
searchResponse -> {
|
||||
List<DataFrameTransformStateAndStats> stats = new ArrayList<>();
|
||||
List<DataFrameTransformStoredDoc> stats = new ArrayList<>();
|
||||
for (SearchHit hit : searchResponse.getHits().getHits()) {
|
||||
BytesReference source = hit.getSourceRef();
|
||||
try (InputStream stream = source.streamInput();
|
||||
XContentParser parser = XContentFactory.xContent(XContentType.JSON)
|
||||
.createParser(NamedXContentRegistry.EMPTY, LoggingDeprecationHandler.INSTANCE, stream)) {
|
||||
stats.add(DataFrameTransformStateAndStats.fromXContent(parser));
|
||||
stats.add(DataFrameTransformStoredDoc.fromXContent(parser));
|
||||
} catch (IOException e) {
|
||||
listener.onFailure(
|
||||
new ElasticsearchParseException("failed to parse data frame stats from search hit", e));
|
||||
|
@ -34,7 +34,7 @@ import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransform;
|
||||
import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformCheckpoint;
|
||||
import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformConfig;
|
||||
import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformState;
|
||||
import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformStateAndStats;
|
||||
import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformStoredDoc;
|
||||
import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformTaskState;
|
||||
import org.elasticsearch.xpack.core.indexing.IndexerState;
|
||||
import org.elasticsearch.xpack.core.scheduler.SchedulerEngine;
|
||||
@ -185,7 +185,7 @@ public class DataFrameTransformPersistentTasksExecutor extends PersistentTasksEx
|
||||
// <3> Set the previous stats (if they exist), initialize the indexer, start the task (If it is STOPPED)
|
||||
// Since we don't create the task until `_start` is called, if we see that the task state is stopped, attempt to start
|
||||
// Schedule execution regardless
|
||||
ActionListener<DataFrameTransformStateAndStats> transformStatsActionListener = ActionListener.wrap(
|
||||
ActionListener<DataFrameTransformStoredDoc> transformStatsActionListener = ActionListener.wrap(
|
||||
stateAndStats -> {
|
||||
logger.trace("[{}] initializing state and stats: [{}]", transformId, stateAndStats.toString());
|
||||
indexerBuilder.setInitialStats(stateAndStats.getTransformStats())
|
||||
@ -224,7 +224,7 @@ public class DataFrameTransformPersistentTasksExecutor extends PersistentTasksEx
|
||||
ActionListener<Map<String, String>> getFieldMappingsListener = ActionListener.wrap(
|
||||
fieldMappings -> {
|
||||
indexerBuilder.setFieldMappings(fieldMappings);
|
||||
transformsConfigManager.getTransformStats(transformId, transformStatsActionListener);
|
||||
transformsConfigManager.getTransformStoredDoc(transformId, transformStatsActionListener);
|
||||
},
|
||||
error -> {
|
||||
String msg = DataFrameMessages.getMessage(DataFrameMessages.DATA_FRAME_UNABLE_TO_GATHER_FIELD_MAPPINGS,
|
||||
|
@ -35,11 +35,10 @@ import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameIndexerPositio
|
||||
import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameIndexerTransformStats;
|
||||
import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransform;
|
||||
import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformCheckpoint;
|
||||
import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformCheckpointingInfo;
|
||||
import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformConfig;
|
||||
import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformProgress;
|
||||
import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformState;
|
||||
import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformStateAndStats;
|
||||
import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformStoredDoc;
|
||||
import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformTaskState;
|
||||
import org.elasticsearch.xpack.core.dataframe.utils.ExceptionsHelper;
|
||||
import org.elasticsearch.xpack.core.indexing.IndexerState;
|
||||
@ -691,11 +690,11 @@ public class DataFrameTransformTask extends AllocatedPersistentTask implements S
|
||||
getProgress());
|
||||
logger.debug("Updating persistent state of transform [{}] to [{}]", transformConfig.getId(), state.toString());
|
||||
|
||||
// Persisting stats when we call `doSaveState` should be ok as we only call it on a state transition and
|
||||
// only every-so-often when doing the bulk indexing calls. See AsyncTwoPhaseIndexer#onBulkResponse for current periodicity
|
||||
transformsConfigManager.putOrUpdateTransformStats(
|
||||
new DataFrameTransformStateAndStats(transformId, state, getStats(),
|
||||
DataFrameTransformCheckpointingInfo.EMPTY), // TODO should this be null
|
||||
// Persist the current state and stats in the internal index. The interval of this method being
|
||||
// called is controlled by AsyncTwoPhaseIndexer#onBulkResponse which calls doSaveState every so
|
||||
// often when doing bulk indexing calls or at the end of one indexing run.
|
||||
transformsConfigManager.putOrUpdateTransformStoredDoc(
|
||||
new DataFrameTransformStoredDoc(transformId, state, getStats()),
|
||||
ActionListener.wrap(
|
||||
r -> {
|
||||
// for auto stop shutdown the task
|
||||
|
@ -39,10 +39,15 @@ import org.elasticsearch.index.store.StoreStats;
|
||||
import org.elasticsearch.index.warmer.WarmerStats;
|
||||
import org.elasticsearch.search.suggest.completion.CompletionStats;
|
||||
import org.elasticsearch.test.client.NoOpClient;
|
||||
import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameIndexerPosition;
|
||||
import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameIndexerPositionTests;
|
||||
import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformCheckpoint;
|
||||
import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformCheckpointStats;
|
||||
import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformCheckpointingInfo;
|
||||
import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformConfigTests;
|
||||
import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformProgress;
|
||||
import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformProgressTests;
|
||||
import org.elasticsearch.xpack.core.indexing.IndexerState;
|
||||
import org.elasticsearch.xpack.dataframe.DataFrameSingleNodeTestCase;
|
||||
import org.elasticsearch.xpack.dataframe.persistence.DataFrameTransformsConfigManager;
|
||||
import org.junit.After;
|
||||
@ -75,7 +80,7 @@ public class DataFrameTransformCheckpointServiceNodeTests extends DataFrameSingl
|
||||
super(testName);
|
||||
}
|
||||
|
||||
public void setShardStats(ShardStats[] shardStats) {
|
||||
void setShardStats(ShardStats[] shardStats) {
|
||||
this.shardStats = shardStats;
|
||||
|
||||
Set<String> indices = new HashSet<>();
|
||||
@ -172,6 +177,8 @@ public class DataFrameTransformCheckpointServiceNodeTests extends DataFrameSingl
|
||||
public void testGetCheckpointStats() throws InterruptedException {
|
||||
String transformId = randomAlphaOfLengthBetween(3, 10);
|
||||
long timestamp = 1000;
|
||||
DataFrameIndexerPosition position = DataFrameIndexerPositionTests.randomDataFrameIndexerPosition();
|
||||
DataFrameTransformProgress progress = DataFrameTransformProgressTests.randomDataFrameTransformProgress();
|
||||
|
||||
// create transform
|
||||
assertAsync(
|
||||
@ -191,26 +198,32 @@ public class DataFrameTransformCheckpointServiceNodeTests extends DataFrameSingl
|
||||
|
||||
mockClientForCheckpointing.setShardStats(createShardStats(createCheckPointMap(transformId, 20, 20, 20)));
|
||||
DataFrameTransformCheckpointingInfo checkpointInfo = new DataFrameTransformCheckpointingInfo(
|
||||
new DataFrameTransformCheckpointStats(timestamp, 0L),
|
||||
new DataFrameTransformCheckpointStats(timestamp + 100L, 0L),
|
||||
new DataFrameTransformCheckpointStats(1, null, null, null, timestamp, 0L),
|
||||
new DataFrameTransformCheckpointStats(2, IndexerState.STARTED, position, progress, timestamp + 100L, 0L),
|
||||
30L);
|
||||
|
||||
assertAsync(listener -> transformsCheckpointService.getCheckpointStats(transformId, 1, 2, listener), checkpointInfo, null, null);
|
||||
assertAsync(listener ->
|
||||
transformsCheckpointService.getCheckpointStats(transformId, 1, 2, IndexerState.STARTED, position, progress, listener),
|
||||
checkpointInfo, null, null);
|
||||
|
||||
mockClientForCheckpointing.setShardStats(createShardStats(createCheckPointMap(transformId, 10, 50, 33)));
|
||||
checkpointInfo = new DataFrameTransformCheckpointingInfo(
|
||||
new DataFrameTransformCheckpointStats(timestamp, 0L),
|
||||
new DataFrameTransformCheckpointStats(timestamp + 100L, 0L),
|
||||
new DataFrameTransformCheckpointStats(1, null, null, null, timestamp, 0L),
|
||||
new DataFrameTransformCheckpointStats(2, IndexerState.INDEXING, position, progress, timestamp + 100L, 0L),
|
||||
63L);
|
||||
assertAsync(listener -> transformsCheckpointService.getCheckpointStats(transformId, 1, 2, listener), checkpointInfo, null, null);
|
||||
assertAsync(listener ->
|
||||
transformsCheckpointService.getCheckpointStats(transformId, 1, 2, IndexerState.INDEXING, position, progress, listener),
|
||||
checkpointInfo, null, null);
|
||||
|
||||
// same as current
|
||||
mockClientForCheckpointing.setShardStats(createShardStats(createCheckPointMap(transformId, 10, 10, 10)));
|
||||
checkpointInfo = new DataFrameTransformCheckpointingInfo(
|
||||
new DataFrameTransformCheckpointStats(timestamp, 0L),
|
||||
new DataFrameTransformCheckpointStats(timestamp + 100L, 0L),
|
||||
new DataFrameTransformCheckpointStats(1, null, null, null, timestamp, 0L),
|
||||
new DataFrameTransformCheckpointStats(2, IndexerState.STOPPING, position, progress, timestamp + 100L, 0L),
|
||||
0L);
|
||||
assertAsync(listener -> transformsCheckpointService.getCheckpointStats(transformId, 1, 2, listener), checkpointInfo, null, null);
|
||||
assertAsync(listener ->
|
||||
transformsCheckpointService.getCheckpointStats(transformId, 1, 2, IndexerState.STOPPING, position, progress, listener),
|
||||
checkpointInfo, null, null);
|
||||
}
|
||||
|
||||
private static Map<String, long[]> createCheckPointMap(String index, long checkpointShard1, long checkpointShard2,
|
||||
|
@ -35,7 +35,6 @@ import org.elasticsearch.test.ESTestCase;
|
||||
|
||||
import java.nio.file.Path;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
@ -61,7 +60,7 @@ public class DataFrameTransformsCheckpointServiceTests extends ESTestCase {
|
||||
|
||||
// low-level compare
|
||||
for (Entry<String, long[]> entry : expectedCheckpoints.entrySet()) {
|
||||
assertTrue(Arrays.equals(entry.getValue(), checkpoints.get(entry.getKey())));
|
||||
assertArrayEquals(entry.getValue(), checkpoints.get(entry.getKey()));
|
||||
}
|
||||
}
|
||||
|
||||
@ -78,7 +77,7 @@ public class DataFrameTransformsCheckpointServiceTests extends ESTestCase {
|
||||
|
||||
// low-level compare
|
||||
for (Entry<String, long[]> entry : expectedCheckpoints.entrySet()) {
|
||||
assertTrue(Arrays.equals(entry.getValue(), checkpoints.get(entry.getKey())));
|
||||
assertArrayEquals(entry.getValue(), checkpoints.get(entry.getKey()));
|
||||
}
|
||||
}
|
||||
|
||||
@ -95,7 +94,7 @@ public class DataFrameTransformsCheckpointServiceTests extends ESTestCase {
|
||||
|
||||
// low-level compare
|
||||
for (Entry<String, long[]> entry : expectedCheckpoints.entrySet()) {
|
||||
assertTrue(Arrays.equals(entry.getValue(), checkpoints.get(entry.getKey())));
|
||||
assertArrayEquals(entry.getValue(), checkpoints.get(entry.getKey()));
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -15,8 +15,8 @@ import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformCheck
|
||||
import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformCheckpointTests;
|
||||
import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformConfig;
|
||||
import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformConfigTests;
|
||||
import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformStateAndStats;
|
||||
import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformStateAndStatsTests;
|
||||
import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformStoredDoc;
|
||||
import org.elasticsearch.xpack.core.dataframe.transforms.DataFrameTransformStoredDocTests;
|
||||
import org.elasticsearch.xpack.dataframe.DataFrameSingleNodeTestCase;
|
||||
import org.junit.Before;
|
||||
|
||||
@ -245,39 +245,37 @@ public class DataFrameTransformsConfigManagerTests extends DataFrameSingleNodeTe
|
||||
|
||||
}
|
||||
|
||||
public void testStateAndStats() throws InterruptedException {
|
||||
String transformId = "transform_test_stats_create_read_update";
|
||||
public void testStoredDoc() throws InterruptedException {
|
||||
String transformId = "transform_test_stored_doc_create_read_update";
|
||||
|
||||
DataFrameTransformStateAndStats stateAndStats =
|
||||
DataFrameTransformStateAndStatsTests.randomDataFrameTransformStateAndStats(transformId);
|
||||
DataFrameTransformStoredDoc storedDocs = DataFrameTransformStoredDocTests.randomDataFrameTransformStoredDoc(transformId);
|
||||
|
||||
assertAsync(listener -> transformsConfigManager.putOrUpdateTransformStats(stateAndStats, listener), Boolean.TRUE, null, null);
|
||||
assertAsync(listener -> transformsConfigManager.getTransformStats(transformId, listener), stateAndStats, null, null);
|
||||
assertAsync(listener -> transformsConfigManager.putOrUpdateTransformStoredDoc(storedDocs, listener), Boolean.TRUE, null, null);
|
||||
assertAsync(listener -> transformsConfigManager.getTransformStoredDoc(transformId, listener), storedDocs, null, null);
|
||||
|
||||
DataFrameTransformStateAndStats updated =
|
||||
DataFrameTransformStateAndStatsTests.randomDataFrameTransformStateAndStats(transformId);
|
||||
assertAsync(listener -> transformsConfigManager.putOrUpdateTransformStats(updated, listener), Boolean.TRUE, null, null);
|
||||
assertAsync(listener -> transformsConfigManager.getTransformStats(transformId, listener), updated, null, null);
|
||||
DataFrameTransformStoredDoc updated = DataFrameTransformStoredDocTests.randomDataFrameTransformStoredDoc(transformId);
|
||||
assertAsync(listener -> transformsConfigManager.putOrUpdateTransformStoredDoc(updated, listener), Boolean.TRUE, null, null);
|
||||
assertAsync(listener -> transformsConfigManager.getTransformStoredDoc(transformId, listener), updated, null, null);
|
||||
}
|
||||
|
||||
public void testGetStateAndStatsMultiple() throws InterruptedException {
|
||||
public void testGetStoredDocMultiple() throws InterruptedException {
|
||||
int numStats = randomIntBetween(10, 15);
|
||||
List<DataFrameTransformStateAndStats> expectedStats = new ArrayList<>();
|
||||
List<DataFrameTransformStoredDoc> expectedDocs = new ArrayList<>();
|
||||
for (int i=0; i<numStats; i++) {
|
||||
DataFrameTransformStateAndStats stat =
|
||||
DataFrameTransformStateAndStatsTests.randomDataFrameTransformStateAndStats(randomAlphaOfLength(6));
|
||||
expectedStats.add(stat);
|
||||
assertAsync(listener -> transformsConfigManager.putOrUpdateTransformStats(stat, listener), Boolean.TRUE, null, null);
|
||||
DataFrameTransformStoredDoc stat =
|
||||
DataFrameTransformStoredDocTests.randomDataFrameTransformStoredDoc(randomAlphaOfLength(6));
|
||||
expectedDocs.add(stat);
|
||||
assertAsync(listener -> transformsConfigManager.putOrUpdateTransformStoredDoc(stat, listener), Boolean.TRUE, null, null);
|
||||
}
|
||||
|
||||
// remove one of the put stats so we don't retrieve all
|
||||
if (expectedStats.size() > 1) {
|
||||
expectedStats.remove(expectedStats.size() -1);
|
||||
// remove one of the put docs so we don't retrieve all
|
||||
if (expectedDocs.size() > 1) {
|
||||
expectedDocs.remove(expectedDocs.size() - 1);
|
||||
}
|
||||
List<String> ids = expectedStats.stream().map(DataFrameTransformStateAndStats::getId).collect(Collectors.toList());
|
||||
List<String> ids = expectedDocs.stream().map(DataFrameTransformStoredDoc::getId).collect(Collectors.toList());
|
||||
|
||||
// get stats will be ordered by id
|
||||
expectedStats.sort(Comparator.comparing(DataFrameTransformStateAndStats::getId));
|
||||
assertAsync(listener -> transformsConfigManager.getTransformStats(ids, listener), expectedStats, null, null);
|
||||
// returned docs will be ordered by id
|
||||
expectedDocs.sort(Comparator.comparing(DataFrameTransformStoredDoc::getId));
|
||||
assertAsync(listener -> transformsConfigManager.getTransformStoredDoc(ids, listener), expectedDocs, null, null);
|
||||
}
|
||||
}
|
||||
|
@ -619,8 +619,7 @@ setup:
|
||||
transform_id: "airline-transform-start-delete"
|
||||
- match: { count: 1 }
|
||||
- match: { transforms.0.id: "airline-transform-start-delete" }
|
||||
- match: { transforms.0.state.indexer_state: "/started|indexing/" }
|
||||
- match: { transforms.0.state.task_state: "started" }
|
||||
- match: { transforms.0.task_state: "started" }
|
||||
|
||||
- do:
|
||||
catch: /Cannot delete data frame \[airline-transform-start-delete\] as the task is running/
|
||||
|
@ -100,8 +100,7 @@ teardown:
|
||||
transform_id: "airline-transform-start-stop"
|
||||
- match: { count: 1 }
|
||||
- match: { transforms.0.id: "airline-transform-start-stop" }
|
||||
- match: { transforms.0.state.indexer_state: "/started|indexing/" }
|
||||
- match: { transforms.0.state.task_state: "started" }
|
||||
- match: { transforms.0.task_state: "started" }
|
||||
|
||||
- do:
|
||||
data_frame.stop_data_frame_transform:
|
||||
@ -114,8 +113,7 @@ teardown:
|
||||
transform_id: "airline-transform-start-stop"
|
||||
- match: { count: 1 }
|
||||
- match: { transforms.0.id: "airline-transform-start-stop" }
|
||||
- match: { transforms.0.state.indexer_state: "stopped" }
|
||||
- match: { transforms.0.state.task_state: "stopped" }
|
||||
- match: { transforms.0.task_state: "stopped" }
|
||||
|
||||
- do:
|
||||
data_frame.start_data_frame_transform:
|
||||
@ -127,8 +125,7 @@ teardown:
|
||||
transform_id: "airline-transform-start-stop"
|
||||
- match: { count: 1 }
|
||||
- match: { transforms.0.id: "airline-transform-start-stop" }
|
||||
- match: { transforms.0.state.indexer_state: "/started|indexing/" }
|
||||
- match: { transforms.0.state.task_state: "started" }
|
||||
- match: { transforms.0.task_state: "started" }
|
||||
---
|
||||
"Test start/stop/start continuous transform":
|
||||
- do:
|
||||
@ -160,8 +157,7 @@ teardown:
|
||||
transform_id: "airline-transform-start-stop-continuous"
|
||||
- match: { count: 1 }
|
||||
- match: { transforms.0.id: "airline-transform-start-stop-continuous" }
|
||||
- match: { transforms.0.state.indexer_state: "/started|indexing/" }
|
||||
- match: { transforms.0.state.task_state: "started" }
|
||||
- match: { transforms.0.task_state: "started" }
|
||||
|
||||
- do:
|
||||
data_frame.stop_data_frame_transform:
|
||||
@ -174,8 +170,7 @@ teardown:
|
||||
transform_id: "airline-transform-start-stop-continuous"
|
||||
- match: { count: 1 }
|
||||
- match: { transforms.0.id: "airline-transform-start-stop-continuous" }
|
||||
- match: { transforms.0.state.indexer_state: "stopped" }
|
||||
- match: { transforms.0.state.task_state: "stopped" }
|
||||
- match: { transforms.0.task_state: "stopped" }
|
||||
|
||||
- do:
|
||||
data_frame.start_data_frame_transform:
|
||||
@ -187,8 +182,7 @@ teardown:
|
||||
transform_id: "airline-transform-start-stop-continuous"
|
||||
- match: { count: 1 }
|
||||
- match: { transforms.0.id: "airline-transform-start-stop-continuous" }
|
||||
- match: { transforms.0.state.indexer_state: "/started|indexing/" }
|
||||
- match: { transforms.0.state.task_state: "started" }
|
||||
- match: { transforms.0.task_state: "started" }
|
||||
|
||||
- do:
|
||||
data_frame.stop_data_frame_transform:
|
||||
@ -250,16 +244,14 @@ teardown:
|
||||
transform_id: "airline-transform-start-stop"
|
||||
- match: { count: 1 }
|
||||
- match: { transforms.0.id: "airline-transform-start-stop" }
|
||||
- match: { transforms.0.state.indexer_state: "/started|indexing/" }
|
||||
- match: { transforms.0.state.task_state: "started" }
|
||||
- match: { transforms.0.task_state: "started" }
|
||||
|
||||
- do:
|
||||
data_frame.get_data_frame_transform_stats:
|
||||
transform_id: "airline-transform-start-later"
|
||||
- match: { count: 1 }
|
||||
- match: { transforms.0.id: "airline-transform-start-later" }
|
||||
- match: { transforms.0.state.indexer_state: "stopped" }
|
||||
- match: { transforms.0.state.task_state: "stopped" }
|
||||
- match: { transforms.0.task_state: "stopped" }
|
||||
|
||||
- do:
|
||||
data_frame.start_data_frame_transform:
|
||||
@ -278,8 +270,7 @@ teardown:
|
||||
transform_id: "airline-transform-start-later"
|
||||
- match: { count: 1 }
|
||||
- match: { transforms.0.id: "airline-transform-start-later" }
|
||||
- match: { transforms.0.state.indexer_state: "/started|indexing/" }
|
||||
- match: { transforms.0.state.task_state: "started" }
|
||||
- match: { transforms.0.task_state: "started" }
|
||||
|
||||
- do:
|
||||
data_frame.stop_data_frame_transform:
|
||||
@ -325,10 +316,8 @@ teardown:
|
||||
data_frame.get_data_frame_transform_stats:
|
||||
transform_id: "*"
|
||||
- match: { count: 2 }
|
||||
- match: { transforms.0.state.indexer_state: "stopped" }
|
||||
- match: { transforms.0.state.task_state: "stopped" }
|
||||
- match: { transforms.1.state.indexer_state: "stopped" }
|
||||
- match: { transforms.1.state.task_state: "stopped" }
|
||||
- match: { transforms.0.task_state: "stopped" }
|
||||
- match: { transforms.1.task_state: "stopped" }
|
||||
|
||||
- do:
|
||||
data_frame.delete_data_frame_transform:
|
||||
|
@ -47,9 +47,8 @@ teardown:
|
||||
transform_id: "airline-transform-stats"
|
||||
- match: { count: 1 }
|
||||
- match: { transforms.0.id: "airline-transform-stats" }
|
||||
- match: { transforms.0.state.indexer_state: "/started|indexing|stopped/" }
|
||||
- match: { transforms.0.state.task_state: "/started|stopped/" }
|
||||
- lte: { transforms.0.state.checkpoint: 1 }
|
||||
- match: { transforms.0.task_state: "/started|stopped/" }
|
||||
- lte: { transforms.0.checkpointing.last.checkpoint: 1 }
|
||||
- lte: { transforms.0.stats.pages_processed: 1 }
|
||||
- match: { transforms.0.stats.documents_processed: 0 }
|
||||
- match: { transforms.0.stats.documents_indexed: 0 }
|
||||
@ -163,18 +162,14 @@ teardown:
|
||||
transform_id: "*"
|
||||
- match: { count: 2 }
|
||||
- match: { transforms.0.id: "airline-transform-stats" }
|
||||
- match: { transforms.0.state.indexer_state: "/started|indexing|stopped/" }
|
||||
- match: { transforms.1.id: "airline-transform-stats-dos" }
|
||||
- match: { transforms.1.state.indexer_state: "stopped" }
|
||||
|
||||
- do:
|
||||
data_frame.get_data_frame_transform_stats:
|
||||
transform_id: "_all"
|
||||
- match: { count: 2 }
|
||||
- match: { transforms.0.id: "airline-transform-stats" }
|
||||
- match: { transforms.0.state.indexer_state: "/started|indexing|stopped/" }
|
||||
- match: { transforms.1.id: "airline-transform-stats-dos" }
|
||||
- match: { transforms.1.state.indexer_state: "stopped" }
|
||||
|
||||
---
|
||||
"Test get single transform stats when it does not have a task":
|
||||
@ -196,9 +191,8 @@ teardown:
|
||||
transform_id: "airline-transform-stats-dos"
|
||||
- match: { count: 1 }
|
||||
- match: { transforms.0.id: "airline-transform-stats-dos" }
|
||||
- match: { transforms.0.state.indexer_state: "stopped" }
|
||||
- match: { transforms.0.state.checkpoint: 0 }
|
||||
- is_false: transforms.0.state.progress
|
||||
- match: { transforms.0.checkpointing.last.checkpoint: 0 }
|
||||
- is_false: transforms.0.checkpointing.next.progress
|
||||
- match: { transforms.0.stats.pages_processed: 0 }
|
||||
- match: { transforms.0.stats.documents_processed: 0 }
|
||||
- match: { transforms.0.stats.documents_indexed: 0 }
|
||||
@ -232,17 +226,16 @@ teardown:
|
||||
transform_id: "airline-transform-stats-continuous"
|
||||
- match: { count: 1 }
|
||||
- match: { transforms.0.id: "airline-transform-stats-continuous" }
|
||||
- match: { transforms.0.state.indexer_state: "/started|indexing|stopped/" }
|
||||
# Since this is continuous, there is no worry of it automatically stopping
|
||||
- match: { transforms.0.state.task_state: "started" }
|
||||
- lte: { transforms.0.state.checkpoint: 1 }
|
||||
- match: { transforms.0.task_state: "started" }
|
||||
- lte: { transforms.0.checkpointing.last.checkpoint: 1 }
|
||||
# Since this is continuous, and _start does not return until it is assigned
|
||||
# we should see a node assignment
|
||||
- is_true: transforms.0.state.node
|
||||
- is_true: transforms.0.state.node.id
|
||||
- is_true: transforms.0.state.node.name
|
||||
- is_true: transforms.0.state.node.ephemeral_id
|
||||
- is_true: transforms.0.state.node.transport_address
|
||||
- is_true: transforms.0.node
|
||||
- is_true: transforms.0.node.id
|
||||
- is_true: transforms.0.node.name
|
||||
- is_true: transforms.0.node.ephemeral_id
|
||||
- is_true: transforms.0.node.transport_address
|
||||
- lte: { transforms.0.stats.pages_processed: 1 }
|
||||
- match: { transforms.0.stats.documents_processed: 0 }
|
||||
- match: { transforms.0.stats.documents_indexed: 0 }
|
||||
|
@ -0,0 +1,279 @@
|
||||
/*
|
||||
* 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.upgrades;
|
||||
|
||||
import org.apache.http.entity.ContentType;
|
||||
import org.apache.http.entity.StringEntity;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.elasticsearch.Version;
|
||||
import org.elasticsearch.client.Request;
|
||||
import org.elasticsearch.client.Response;
|
||||
import org.elasticsearch.client.dataframe.GetDataFrameTransformStatsResponse;
|
||||
import org.elasticsearch.client.dataframe.transforms.DataFrameTransformConfig;
|
||||
import org.elasticsearch.client.dataframe.transforms.DataFrameTransformStats;
|
||||
import org.elasticsearch.client.dataframe.transforms.DataFrameTransformTaskState;
|
||||
import org.elasticsearch.client.dataframe.transforms.DestConfig;
|
||||
import org.elasticsearch.client.dataframe.transforms.SourceConfig;
|
||||
import org.elasticsearch.client.dataframe.transforms.TimeSyncConfig;
|
||||
import org.elasticsearch.client.dataframe.transforms.pivot.GroupConfig;
|
||||
import org.elasticsearch.client.dataframe.transforms.pivot.PivotConfig;
|
||||
import org.elasticsearch.client.dataframe.transforms.pivot.TermsGroupSource;
|
||||
import org.elasticsearch.common.Booleans;
|
||||
import org.elasticsearch.common.Strings;
|
||||
import org.elasticsearch.common.unit.TimeValue;
|
||||
import org.elasticsearch.common.xcontent.DeprecationHandler;
|
||||
import org.elasticsearch.common.xcontent.NamedXContentRegistry;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.common.xcontent.XContentParser;
|
||||
import org.elasticsearch.common.xcontent.XContentType;
|
||||
import org.elasticsearch.search.aggregations.AggregationBuilders;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactories;
|
||||
import org.elasticsearch.xpack.test.rest.XPackRestTestConstants;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.time.Instant;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
|
||||
import static org.hamcrest.Matchers.equalTo;
|
||||
import static org.hamcrest.Matchers.greaterThan;
|
||||
import static org.hamcrest.Matchers.greaterThanOrEqualTo;
|
||||
import static org.hamcrest.Matchers.hasSize;
|
||||
|
||||
@LuceneTestCase.AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/43662")
|
||||
public class DataFrameSurvivesUpgradeIT extends AbstractUpgradeTestCase {
|
||||
|
||||
private static final Version UPGRADE_FROM_VERSION = Version.fromString(System.getProperty("tests.upgrade_from_version"));
|
||||
private static final String DATAFRAME_ENDPOINT = "/_data_frame/transforms/";
|
||||
private static final String CONTINUOUS_DATA_FRAME_ID = "continuous-data-frame-upgrade-job";
|
||||
private static final String CONTINUOUS_DATA_FRAME_SOURCE = "data-frame-upgrade-continuous-source";
|
||||
private static final List<String> ENTITIES = Stream.iterate(1, n -> n + 1)
|
||||
.limit(5)
|
||||
.map(v -> "user_" + v)
|
||||
.collect(Collectors.toList());
|
||||
private static final List<TimeValue> BUCKETS = Stream.iterate(1, n -> n + 1)
|
||||
.limit(5)
|
||||
.map(TimeValue::timeValueMinutes)
|
||||
.collect(Collectors.toList());
|
||||
|
||||
@Override
|
||||
protected Collection<String> templatesToWaitFor() {
|
||||
return Stream.concat(XPackRestTestConstants.DATA_FRAME_TEMPLATES.stream(),
|
||||
super.templatesToWaitFor().stream()).collect(Collectors.toSet());
|
||||
}
|
||||
|
||||
protected static void waitForPendingDataFrameTasks() throws Exception {
|
||||
waitForPendingTasks(adminClient(), taskName -> taskName.startsWith("data_frame/transforms") == false);
|
||||
}
|
||||
|
||||
/**
|
||||
* The purpose of this test is to ensure that when a job is open through a rolling upgrade we upgrade the results
|
||||
* index mappings when it is assigned to an upgraded node even if no other ML endpoint is called after the upgrade
|
||||
*/
|
||||
public void testDataFramesRollingUpgrade() throws Exception {
|
||||
assumeTrue("Continuous data frames not supported until 7.3", UPGRADE_FROM_VERSION.onOrAfter(Version.V_7_3_0));
|
||||
Request waitForYellow = new Request("GET", "/_cluster/health");
|
||||
waitForYellow.addParameter("wait_for_nodes", "3");
|
||||
waitForYellow.addParameter("wait_for_status", "yellow");
|
||||
switch (CLUSTER_TYPE) {
|
||||
case OLD:
|
||||
createAndStartContinuousDataFrame();
|
||||
break;
|
||||
case MIXED:
|
||||
client().performRequest(waitForYellow);
|
||||
long lastCheckpoint = 1;
|
||||
if (Booleans.parseBoolean(System.getProperty("tests.first_round")) == false) {
|
||||
lastCheckpoint = 2;
|
||||
}
|
||||
verifyContinuousDataFrameHandlesData(lastCheckpoint);
|
||||
break;
|
||||
case UPGRADED:
|
||||
client().performRequest(waitForYellow);
|
||||
verifyContinuousDataFrameHandlesData(3);
|
||||
cleanUpTransforms();
|
||||
break;
|
||||
default:
|
||||
throw new UnsupportedOperationException("Unknown cluster type [" + CLUSTER_TYPE + "]");
|
||||
}
|
||||
}
|
||||
|
||||
private void cleanUpTransforms() throws Exception {
|
||||
stopTransform(CONTINUOUS_DATA_FRAME_ID);
|
||||
deleteTransform(CONTINUOUS_DATA_FRAME_ID);
|
||||
waitForPendingDataFrameTasks();
|
||||
}
|
||||
|
||||
private void createAndStartContinuousDataFrame() throws Exception {
|
||||
createIndex(CONTINUOUS_DATA_FRAME_SOURCE);
|
||||
long totalDocsWritten = 0;
|
||||
for (TimeValue bucket : BUCKETS) {
|
||||
int docs = randomIntBetween(1, 25);
|
||||
putData(CONTINUOUS_DATA_FRAME_SOURCE, docs, bucket, ENTITIES);
|
||||
totalDocsWritten += docs * ENTITIES.size();
|
||||
}
|
||||
|
||||
DataFrameTransformConfig config = DataFrameTransformConfig.builder()
|
||||
.setSyncConfig(new TimeSyncConfig("timestamp", TimeValue.timeValueSeconds(30)))
|
||||
.setPivotConfig(PivotConfig.builder()
|
||||
.setAggregations(new AggregatorFactories.Builder().addAggregator(AggregationBuilders.avg("stars").field("stars")))
|
||||
.setGroups(GroupConfig.builder().groupBy("user_id", TermsGroupSource.builder().setField("user_id").build()).build())
|
||||
.build())
|
||||
.setDest(DestConfig.builder().setIndex(CONTINUOUS_DATA_FRAME_ID + "_idx").build())
|
||||
.setSource(SourceConfig.builder().setIndex(CONTINUOUS_DATA_FRAME_SOURCE).build())
|
||||
.setId(CONTINUOUS_DATA_FRAME_ID)
|
||||
.build();
|
||||
putTransform(CONTINUOUS_DATA_FRAME_ID, config);
|
||||
|
||||
startTransform(CONTINUOUS_DATA_FRAME_ID);
|
||||
waitUntilAfterCheckpoint(CONTINUOUS_DATA_FRAME_ID, 0L);
|
||||
|
||||
DataFrameTransformStats stateAndStats = getTransformStats(CONTINUOUS_DATA_FRAME_ID);
|
||||
|
||||
assertThat(stateAndStats.getIndexerStats().getOutputDocuments(), equalTo((long)ENTITIES.size()));
|
||||
assertThat(stateAndStats.getIndexerStats().getNumDocuments(), equalTo(totalDocsWritten));
|
||||
assertThat(stateAndStats.getTaskState(), equalTo(DataFrameTransformTaskState.STARTED));
|
||||
}
|
||||
|
||||
private void verifyContinuousDataFrameHandlesData(long expectedLastCheckpoint) throws Exception {
|
||||
|
||||
// A continuous data frame should automatically become started when it gets assigned to a node
|
||||
// if it was assigned to the node that was removed from the cluster
|
||||
assertBusy(() -> {
|
||||
DataFrameTransformStats stateAndStats = getTransformStats(CONTINUOUS_DATA_FRAME_ID);
|
||||
assertThat(stateAndStats.getTaskState(), equalTo(DataFrameTransformTaskState.STARTED));
|
||||
},
|
||||
120,
|
||||
TimeUnit.SECONDS);
|
||||
|
||||
DataFrameTransformStats previousStateAndStats = getTransformStats(CONTINUOUS_DATA_FRAME_ID);
|
||||
|
||||
// Add a new user and write data to it
|
||||
// This is so we can have more reliable data counts, as writing to existing entities requires
|
||||
// rescanning the past data
|
||||
List<String> entities = new ArrayList<>(1);
|
||||
entities.add("user_" + ENTITIES.size() + expectedLastCheckpoint);
|
||||
int docs = 5;
|
||||
// Index the data very recently in the past so that the transform sync delay can catch up to reading it in our spin
|
||||
// wait later.
|
||||
putData(CONTINUOUS_DATA_FRAME_SOURCE, docs, TimeValue.timeValueSeconds(1), entities);
|
||||
|
||||
waitUntilAfterCheckpoint(CONTINUOUS_DATA_FRAME_ID, expectedLastCheckpoint);
|
||||
|
||||
assertBusy(() -> assertThat(
|
||||
getTransformStats(CONTINUOUS_DATA_FRAME_ID).getIndexerStats().getNumDocuments(),
|
||||
greaterThanOrEqualTo(docs + previousStateAndStats.getIndexerStats().getNumDocuments())),
|
||||
120,
|
||||
TimeUnit.SECONDS);
|
||||
DataFrameTransformStats stateAndStats = getTransformStats(CONTINUOUS_DATA_FRAME_ID);
|
||||
|
||||
assertThat(stateAndStats.getTaskState(),
|
||||
equalTo(DataFrameTransformTaskState.STARTED));
|
||||
assertThat(stateAndStats.getIndexerStats().getOutputDocuments(),
|
||||
greaterThan(previousStateAndStats.getIndexerStats().getOutputDocuments()));
|
||||
assertThat(stateAndStats.getIndexerStats().getNumDocuments(),
|
||||
greaterThanOrEqualTo(docs + previousStateAndStats.getIndexerStats().getNumDocuments()));
|
||||
}
|
||||
|
||||
private void putTransform(String id, DataFrameTransformConfig config) throws IOException {
|
||||
final Request createDataframeTransformRequest = new Request("PUT", DATAFRAME_ENDPOINT + id);
|
||||
createDataframeTransformRequest.setJsonEntity(Strings.toString(config));
|
||||
Response response = client().performRequest(createDataframeTransformRequest);
|
||||
assertEquals(200, response.getStatusLine().getStatusCode());
|
||||
}
|
||||
|
||||
private void deleteTransform(String id) throws IOException {
|
||||
Response response = client().performRequest(new Request("DELETE", DATAFRAME_ENDPOINT + id));
|
||||
assertEquals(200, response.getStatusLine().getStatusCode());
|
||||
}
|
||||
|
||||
private void startTransform(String id) throws IOException {
|
||||
final Request startDataframeTransformRequest = new Request("POST", DATAFRAME_ENDPOINT + id + "/_start");
|
||||
Response response = client().performRequest(startDataframeTransformRequest);
|
||||
assertEquals(200, response.getStatusLine().getStatusCode());
|
||||
}
|
||||
|
||||
private void stopTransform(String id) throws IOException {
|
||||
final Request stopDataframeTransformRequest = new Request("POST",
|
||||
DATAFRAME_ENDPOINT + id + "/_stop?wait_for_completion=true");
|
||||
Response response = client().performRequest(stopDataframeTransformRequest);
|
||||
assertEquals(200, response.getStatusLine().getStatusCode());
|
||||
}
|
||||
|
||||
private DataFrameTransformStats getTransformStats(String id) throws IOException {
|
||||
final Request getStats = new Request("GET", DATAFRAME_ENDPOINT + id + "/_stats");
|
||||
Response response = client().performRequest(getStats);
|
||||
assertEquals(200, response.getStatusLine().getStatusCode());
|
||||
XContentType xContentType = XContentType.fromMediaTypeOrFormat(response.getEntity().getContentType().getValue());
|
||||
try (XContentParser parser = xContentType.xContent().createParser(
|
||||
NamedXContentRegistry.EMPTY, DeprecationHandler.THROW_UNSUPPORTED_OPERATION,
|
||||
response.getEntity().getContent())) {
|
||||
GetDataFrameTransformStatsResponse resp = GetDataFrameTransformStatsResponse.fromXContent(parser);
|
||||
assertThat(resp.getTransformsStats(), hasSize(1));
|
||||
return resp.getTransformsStats().get(0);
|
||||
}
|
||||
}
|
||||
|
||||
private void waitUntilAfterCheckpoint(String id, long currentCheckpoint) throws Exception {
|
||||
assertBusy(() -> assertThat(getTransformStats(id).getCheckpointingInfo().getNext().getCheckpoint(), greaterThan(currentCheckpoint)),
|
||||
60, TimeUnit.SECONDS);
|
||||
}
|
||||
|
||||
private void createIndex(String indexName) throws IOException {
|
||||
// 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("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", indexName);
|
||||
req.setEntity(entity);
|
||||
client().performRequest(req);
|
||||
}
|
||||
}
|
||||
|
||||
private void putData(String indexName, int numDocs, TimeValue fromTime, List<String> entityIds) throws IOException {
|
||||
long timeStamp = Instant.now().toEpochMilli() - fromTime.getMillis();
|
||||
|
||||
// create index
|
||||
final StringBuilder bulk = new StringBuilder();
|
||||
for (int i = 0; i < numDocs; i++) {
|
||||
for (String entity : entityIds) {
|
||||
bulk.append("{\"index\":{\"_index\":\"" + indexName + "\"}}\n")
|
||||
.append("{\"user_id\":\"")
|
||||
.append(entity)
|
||||
.append("\",\"stars\":")
|
||||
.append(randomLongBetween(0, 5))
|
||||
.append(",\"timestamp\":")
|
||||
.append(timeStamp)
|
||||
.append("}\n");
|
||||
}
|
||||
}
|
||||
bulk.append("\r\n");
|
||||
final Request bulkRequest = new Request("POST", "/_bulk");
|
||||
bulkRequest.addParameter("refresh", "true");
|
||||
bulkRequest.setJsonEntity(bulk.toString());
|
||||
entityAsMap(client().performRequest(bulkRequest));
|
||||
}
|
||||
}
|
@ -29,7 +29,10 @@
|
||||
transform_id: "mixed-simple-transform"
|
||||
- match: { count: 1 }
|
||||
- match: { transforms.0.id: "mixed-simple-transform" }
|
||||
- match: { transforms.0.state.task_state: "/started|stopped/" }
|
||||
# Since we are breaking the stats format between 7.3 and 7.4 (allowed because we're beta) we
|
||||
# cannot assert on task_state in the mixed cluster as it could be at the top level or under state
|
||||
# TODO: uncomment this assertion in master
|
||||
#- match: { transforms.0.task_state: "/started|stopped/" }
|
||||
|
||||
- do:
|
||||
data_frame.stop_data_frame_transform:
|
||||
@ -42,8 +45,10 @@
|
||||
transform_id: "mixed-simple-transform"
|
||||
- match: { count: 1 }
|
||||
- match: { transforms.0.id: "mixed-simple-transform" }
|
||||
- match: { transforms.0.state.indexer_state: "stopped" }
|
||||
- match: { transforms.0.state.task_state: "stopped" }
|
||||
# Since we are breaking the stats format between 7.3 and 7.4 (allowed because we're beta) we
|
||||
# cannot assert on task_state in the mixed cluster as it could be at the top level or under state
|
||||
# TODO: uncomment this assertion in master
|
||||
#- match: { transforms.0.task_state: "stopped" }
|
||||
|
||||
- do:
|
||||
data_frame.put_data_frame_transform:
|
||||
@ -87,7 +92,10 @@
|
||||
transform_id: "mixed-complex-transform"
|
||||
- match: { count: 1 }
|
||||
- match: { transforms.0.id: "mixed-complex-transform" }
|
||||
- match: { transforms.0.state.task_state: "/started|stopped/" }
|
||||
# Since we are breaking the stats format between 7.3 and 7.4 (allowed because we're beta) we
|
||||
# cannot assert on task_state in the mixed cluster as it could be at the top level or under state
|
||||
# TODO: uncomment this assertion in master
|
||||
#- match: { transforms.0.task_state: "/started|stopped/" }
|
||||
|
||||
- do:
|
||||
data_frame.stop_data_frame_transform:
|
||||
@ -100,8 +108,10 @@
|
||||
transform_id: "mixed-complex-transform"
|
||||
- match: { count: 1 }
|
||||
- match: { transforms.0.id: "mixed-complex-transform" }
|
||||
- match: { transforms.0.state.indexer_state: "stopped" }
|
||||
- match: { transforms.0.state.task_state: "stopped" }
|
||||
# Since we are breaking the stats format between 7.3 and 7.4 (allowed because we're beta) we
|
||||
# cannot assert on task_state in the mixed cluster as it could be at the top level or under state
|
||||
# TODO: uncomment this assertion in master
|
||||
#- match: { transforms.0.task_state: "stopped" }
|
||||
|
||||
---
|
||||
"Test GET, start, and stop old cluster batch transforms":
|
||||
@ -130,7 +140,10 @@
|
||||
transform_id: "old-simple-transform"
|
||||
- match: { count: 1 }
|
||||
- match: { transforms.0.id: "old-simple-transform" }
|
||||
- match: { transforms.0.state.task_state: "/started|stopped/" }
|
||||
# Since we are breaking the stats format between 7.3 and 7.4 (allowed because we're beta) we
|
||||
# cannot assert on task_state in the mixed cluster as it could be at the top level or under state
|
||||
# TODO: uncomment this assertion in master
|
||||
#- match: { transforms.0.task_state: "/started|stopped/" }
|
||||
|
||||
- do:
|
||||
data_frame.stop_data_frame_transform:
|
||||
@ -142,8 +155,10 @@
|
||||
transform_id: "old-simple-transform"
|
||||
- match: { count: 1 }
|
||||
- match: { transforms.0.id: "old-simple-transform" }
|
||||
- match: { transforms.0.state.indexer_state: "stopped" }
|
||||
- match: { transforms.0.state.task_state: "stopped" }
|
||||
# Since we are breaking the stats format between 7.3 and 7.4 (allowed because we're beta) we
|
||||
# cannot assert on task_state in the mixed cluster as it could be at the top level or under state
|
||||
# TODO: uncomment this assertion in master
|
||||
#- match: { transforms.0.task_state: "stopped" }
|
||||
|
||||
- do:
|
||||
data_frame.get_data_frame_transform:
|
||||
@ -166,7 +181,10 @@
|
||||
transform_id: "old-complex-transform"
|
||||
- match: { count: 1 }
|
||||
- match: { transforms.0.id: "old-complex-transform" }
|
||||
- match: { transforms.0.state.task_state: "/started|stopped/" }
|
||||
# Since we are breaking the stats format between 7.3 and 7.4 (allowed because we're beta) we
|
||||
# cannot assert on task_state in the mixed cluster as it could be at the top level or under state
|
||||
# TODO: uncomment this assertion in master
|
||||
#- match: { transforms.0.task_state: "/started|stopped/" }
|
||||
|
||||
- do:
|
||||
data_frame.stop_data_frame_transform:
|
||||
@ -178,5 +196,7 @@
|
||||
transform_id: "old-complex-transform"
|
||||
- match: { count: 1 }
|
||||
- match: { transforms.0.id: "old-complex-transform" }
|
||||
- match: { transforms.0.state.indexer_state: "stopped" }
|
||||
- match: { transforms.0.state.task_state: "stopped" }
|
||||
# Since we are breaking the stats format between 7.3 and 7.4 (allowed because we're beta) we
|
||||
# cannot assert on task_state in the mixed cluster as it could be at the top level or under state
|
||||
# TODO: uncomment this assertion in master
|
||||
#- match: { transforms.0.task_state: "stopped" }
|
||||
|
@ -49,7 +49,6 @@
|
||||
transform_id: "old-simple-transform"
|
||||
- match: { count: 1 }
|
||||
- match: { transforms.0.id: "old-simple-transform" }
|
||||
- match: { transforms.0.state.task_state: "/started|stopped/" }
|
||||
|
||||
- do:
|
||||
data_frame.stop_data_frame_transform:
|
||||
@ -62,8 +61,6 @@
|
||||
transform_id: "old-simple-transform"
|
||||
- match: { count: 1 }
|
||||
- match: { transforms.0.id: "old-simple-transform" }
|
||||
- match: { transforms.0.state.indexer_state: "stopped" }
|
||||
- match: { transforms.0.state.task_state: "stopped" }
|
||||
|
||||
- do:
|
||||
data_frame.put_data_frame_transform:
|
||||
@ -107,7 +104,6 @@
|
||||
transform_id: "old-complex-transform"
|
||||
- match: { count: 1 }
|
||||
- match: { transforms.0.id: "old-complex-transform" }
|
||||
- match: { transforms.0.state.task_state: "/started|stopped/" }
|
||||
|
||||
- do:
|
||||
data_frame.stop_data_frame_transform:
|
||||
@ -120,5 +116,3 @@
|
||||
transform_id: "old-complex-transform"
|
||||
- match: { count: 1 }
|
||||
- match: { transforms.0.id: "old-complex-transform" }
|
||||
- match: { transforms.0.state.indexer_state: "stopped" }
|
||||
- match: { transforms.0.state.task_state: "stopped" }
|
||||
|
@ -27,7 +27,7 @@ setup:
|
||||
transform_id: "old-simple-transform"
|
||||
- match: { count: 1 }
|
||||
- match: { transforms.0.id: "old-simple-transform" }
|
||||
- match: { transforms.0.state.task_state: "/started|stopped/" }
|
||||
- match: { transforms.0.task_state: "/started|stopped/" }
|
||||
|
||||
- do:
|
||||
data_frame.stop_data_frame_transform:
|
||||
@ -39,8 +39,7 @@ setup:
|
||||
transform_id: "old-simple-transform"
|
||||
- match: { count: 1 }
|
||||
- match: { transforms.0.id: "old-simple-transform" }
|
||||
- match: { transforms.0.state.indexer_state: "stopped" }
|
||||
- match: { transforms.0.state.task_state: "stopped" }
|
||||
- match: { transforms.0.task_state: "stopped" }
|
||||
- do:
|
||||
data_frame.get_data_frame_transform:
|
||||
transform_id: "old-complex-transform"
|
||||
@ -62,7 +61,7 @@ setup:
|
||||
transform_id: "old-complex-transform"
|
||||
- match: { count: 1 }
|
||||
- match: { transforms.0.id: "old-complex-transform" }
|
||||
- match: { transforms.0.state.task_state: "/started|stopped/" }
|
||||
- match: { transforms.0.task_state: "/started|stopped/" }
|
||||
|
||||
- do:
|
||||
data_frame.stop_data_frame_transform:
|
||||
@ -74,8 +73,7 @@ setup:
|
||||
transform_id: "old-complex-transform"
|
||||
- match: { count: 1 }
|
||||
- match: { transforms.0.id: "old-complex-transform" }
|
||||
- match: { transforms.0.state.indexer_state: "stopped" }
|
||||
- match: { transforms.0.state.task_state: "stopped" }
|
||||
- match: { transforms.0.task_state: "stopped" }
|
||||
|
||||
# Simple and complex Mixed cluster transforms
|
||||
- do:
|
||||
@ -97,7 +95,7 @@ setup:
|
||||
transform_id: "mixed-simple-transform"
|
||||
- match: { count: 1 }
|
||||
- match: { transforms.0.id: "mixed-simple-transform" }
|
||||
- match: { transforms.0.state.task_state: "/started|stopped/" }
|
||||
- match: { transforms.0.task_state: "/started|stopped/" }
|
||||
|
||||
- do:
|
||||
data_frame.stop_data_frame_transform:
|
||||
@ -109,8 +107,7 @@ setup:
|
||||
transform_id: "mixed-simple-transform"
|
||||
- match: { count: 1 }
|
||||
- match: { transforms.0.id: "mixed-simple-transform" }
|
||||
- match: { transforms.0.state.indexer_state: "stopped" }
|
||||
- match: { transforms.0.state.task_state: "stopped" }
|
||||
- match: { transforms.0.task_state: "stopped" }
|
||||
|
||||
- do:
|
||||
data_frame.get_data_frame_transform:
|
||||
@ -133,7 +130,7 @@ setup:
|
||||
transform_id: "mixed-complex-transform"
|
||||
- match: { count: 1 }
|
||||
- match: { transforms.0.id: "mixed-complex-transform" }
|
||||
- match: { transforms.0.state.task_state: "/started|stopped/" }
|
||||
- match: { transforms.0.task_state: "/started|stopped/" }
|
||||
|
||||
- do:
|
||||
data_frame.stop_data_frame_transform:
|
||||
@ -145,8 +142,7 @@ setup:
|
||||
transform_id: "mixed-complex-transform"
|
||||
- match: { count: 1 }
|
||||
- match: { transforms.0.id: "mixed-complex-transform" }
|
||||
- match: { transforms.0.state.indexer_state: "stopped" }
|
||||
- match: { transforms.0.state.task_state: "stopped" }
|
||||
- match: { transforms.0.task_state: "stopped" }
|
||||
|
||||
# Delete all old and mixed transforms
|
||||
- do:
|
||||
|
Loading…
x
Reference in New Issue
Block a user