Execute API to support default trigger
Until today it was required to provide a trigger event as part of the execute API. There are two issues with it: 1. It's not user friendly (you'd expect that the execute API would just work with just pointing to the watch) 2. The API could expose inconsistencies where on one hand it points to a watch (with a well defined trigger) on the other it enabled the user to provide completely different trigger (of a different type) This change enable supporting default triggers by enabling the trigger engine to create a simulated trigger event. This enables the execute API to look up the trigger type of the pointed watch, and ask the trigger service to simulate an event for it. It is still possible to override the trigger event data by providing it via the `trigger_data` parameter in the API. This simplifies the execute API and prepares it for future trigger types as well. - This commit add missing integration tests for the execute API - Also, removed unused `setIgnoreThrottling` from the execute request/builder. Original commit: elastic/x-pack-elasticsearch@b494ae62e6
This commit is contained in:
parent
2cff8c4c32
commit
d899c4b522
|
@ -59,11 +59,9 @@
|
|||
id: "my_exe_watch"
|
||||
body: >
|
||||
{
|
||||
"trigger_event" : {
|
||||
"schedule" : {
|
||||
"trigger_data" : {
|
||||
"scheduled_time" : "2015-05-05T20:58:02.443Z",
|
||||
"triggered_time" : "2015-05-05T20:58:02.443Z"
|
||||
}
|
||||
},
|
||||
"alternative_input" : {
|
||||
"foo" : "bar"
|
||||
|
|
|
@ -33,15 +33,6 @@
|
|||
- do:
|
||||
watcher.execute_watch:
|
||||
id: "my_logging_watch"
|
||||
body: >
|
||||
{
|
||||
"trigger_event" : {
|
||||
"schedule" : {
|
||||
"scheduled_time" : "2015-05-05T20:58:02.443Z",
|
||||
"triggered_time" : "2015-05-05T20:58:02.443Z"
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
- match: { "watch_record.watch_id": "my_logging_watch" }
|
||||
- match: { "watch_record.state": "executed" }
|
||||
|
|
|
@ -100,11 +100,11 @@ public class WatchRecord implements ToXContent {
|
|||
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
|
||||
builder.startObject();
|
||||
builder.field(Field.WATCH_ID.getPreferredName(), id.watchId());
|
||||
builder.field(Field.STATE.getPreferredName(), state.id());
|
||||
|
||||
builder.field(Field.TRIGGER_EVENT.getPreferredName());
|
||||
triggerEvent.recordXContent(builder, params);
|
||||
|
||||
builder.field(Field.STATE.getPreferredName(), state.id());
|
||||
if (input != null) {
|
||||
builder.startObject(Watch.Field.INPUT.getPreferredName())
|
||||
.field(input.type(), input, params)
|
||||
|
|
|
@ -22,7 +22,6 @@ import org.elasticsearch.watcher.rest.WatcherRestHandler;
|
|||
import org.elasticsearch.watcher.transport.actions.execute.ExecuteWatchRequest;
|
||||
import org.elasticsearch.watcher.transport.actions.execute.ExecuteWatchRequestBuilder;
|
||||
import org.elasticsearch.watcher.transport.actions.execute.ExecuteWatchResponse;
|
||||
import org.elasticsearch.watcher.trigger.TriggerEvent;
|
||||
import org.elasticsearch.watcher.trigger.TriggerService;
|
||||
|
||||
import java.io.IOException;
|
||||
|
@ -61,10 +60,9 @@ public class RestExecuteWatchAction extends WatcherRestHandler {
|
|||
private ExecuteWatchRequest parseRequest(RestRequest request, WatcherClient client) throws IOException {
|
||||
String watchId = request.param("id");
|
||||
ExecuteWatchRequestBuilder builder = client.prepareExecuteWatch(watchId);
|
||||
TriggerEvent triggerEvent = null;
|
||||
|
||||
if (request.content() == null || request.content().length() == 0) {
|
||||
throw new WatcherException("could not parse watch execution request for [{}]. missing required [{}] field.", watchId, Field.TRIGGER_EVENT.getPreferredName());
|
||||
return builder.request();
|
||||
}
|
||||
|
||||
XContentParser parser = XContentHelper.createParser(request.content());
|
||||
|
@ -86,9 +84,8 @@ public class RestExecuteWatchAction extends WatcherRestHandler {
|
|||
} else if (token == XContentParser.Token.START_OBJECT) {
|
||||
if (Field.ALTERNATIVE_INPUT.match(currentFieldName)) {
|
||||
builder.setAlternativeInput(parser.map());
|
||||
} else if (Field.TRIGGER_EVENT.match(currentFieldName)) {
|
||||
triggerEvent = triggerService.parseTriggerEvent(watchId, watchId, parser);
|
||||
builder.setTriggerEvent(triggerEvent);
|
||||
} else if (Field.TRIGGER_DATA.match(currentFieldName)) {
|
||||
builder.setTriggerData(parser.map());
|
||||
} else if (Field.ACTION_MODES.match(currentFieldName)) {
|
||||
while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
|
||||
if (token == XContentParser.Token.FIELD_NAME) {
|
||||
|
@ -112,10 +109,6 @@ public class RestExecuteWatchAction extends WatcherRestHandler {
|
|||
}
|
||||
}
|
||||
|
||||
if (triggerEvent == null) {
|
||||
throw new WatcherException("could not parse watch execution request for [{}]. missing required [{}] field.", watchId, Field.TRIGGER_EVENT.getPreferredName());
|
||||
}
|
||||
|
||||
return builder.request();
|
||||
}
|
||||
|
||||
|
@ -138,6 +131,6 @@ public class RestExecuteWatchAction extends WatcherRestHandler {
|
|||
ParseField ACTION_MODES = new ParseField("action_modes");
|
||||
ParseField ALTERNATIVE_INPUT = new ParseField("alternative_input");
|
||||
ParseField IGNORE_CONDITION = new ParseField("ignore_condition");
|
||||
ParseField TRIGGER_EVENT = new ParseField("trigger_event");
|
||||
ParseField TRIGGER_DATA = new ParseField("trigger_data");
|
||||
}
|
||||
}
|
||||
|
|
|
@ -33,6 +33,19 @@ public class WatcherDateTimeUtils {
|
|||
private WatcherDateTimeUtils() {
|
||||
}
|
||||
|
||||
public static DateTime convertToDate(Object value, Clock clock) {
|
||||
if (value instanceof DateTime) {
|
||||
return (DateTime) value;
|
||||
}
|
||||
if (value instanceof String) {
|
||||
return parseDateMath((String) value, DateTimeZone.UTC, clock);
|
||||
}
|
||||
if (value instanceof Number) {
|
||||
return new DateTime(((Number) value).longValue(), DateTimeZone.UTC);
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
public static DateTime parseDate(String dateAsText) {
|
||||
return parseDate(dateAsText, null);
|
||||
}
|
||||
|
@ -154,6 +167,7 @@ public class WatcherDateTimeUtils {
|
|||
}
|
||||
|
||||
private static class ClockNowCallable implements Callable<Long> {
|
||||
|
||||
private final Clock clock;
|
||||
|
||||
ClockNowCallable(Clock clock){
|
||||
|
|
|
@ -20,6 +20,8 @@ public interface Clock {
|
|||
|
||||
DateTime now();
|
||||
|
||||
DateTime nowUTC();
|
||||
|
||||
DateTime now(DateTimeZone timeZone);
|
||||
|
||||
TimeValue timeElapsedSince(DateTime time);
|
||||
|
|
|
@ -34,6 +34,11 @@ public final class SystemClock implements Clock {
|
|||
return now(DateTimeZone.getDefault());
|
||||
}
|
||||
|
||||
@Override
|
||||
public DateTime nowUTC() {
|
||||
return now(DateTimeZone.UTC);
|
||||
}
|
||||
|
||||
@Override
|
||||
public DateTime now(DateTimeZone timeZone) {
|
||||
return DateTime.now(timeZone);
|
||||
|
|
|
@ -8,15 +8,12 @@ package org.elasticsearch.watcher.transport.actions.execute;
|
|||
import org.elasticsearch.action.ActionRequestValidationException;
|
||||
import org.elasticsearch.action.ValidateActions;
|
||||
import org.elasticsearch.action.support.master.MasterNodeOperationRequest;
|
||||
import org.elasticsearch.common.bytes.BytesReference;
|
||||
import org.elasticsearch.common.Nullable;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.xcontent.ToXContent;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.common.xcontent.XContentFactory;
|
||||
import org.elasticsearch.watcher.execution.ActionExecutionMode;
|
||||
import org.elasticsearch.watcher.support.validation.Validation;
|
||||
import org.elasticsearch.watcher.trigger.TriggerEvent;
|
||||
import org.elasticsearch.watcher.execution.ActionExecutionMode;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.HashMap;
|
||||
|
@ -29,11 +26,9 @@ public class ExecuteWatchRequest extends MasterNodeOperationRequest<ExecuteWatch
|
|||
|
||||
private String id;
|
||||
private boolean ignoreCondition = false;
|
||||
private boolean ignoreThrottle = false;
|
||||
private boolean recordExecution = false;
|
||||
private Map<String, Object> alternativeInput = null;
|
||||
private BytesReference triggerSource = null;
|
||||
private String triggerType = null;
|
||||
private @Nullable Map<String, Object> triggerData = null;
|
||||
private @Nullable Map<String, Object> alternativeInput = null;
|
||||
private Map<String, ActionExecutionMode> actionModes = new HashMap<>();
|
||||
|
||||
ExecuteWatchRequest() {
|
||||
|
@ -74,20 +69,6 @@ public class ExecuteWatchRequest extends MasterNodeOperationRequest<ExecuteWatch
|
|||
this.ignoreCondition = ignoreCondition;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return Should the throttle be ignored for this execution
|
||||
*/
|
||||
public boolean isIgnoreThrottle() {
|
||||
return ignoreThrottle;
|
||||
}
|
||||
|
||||
/**
|
||||
* @param ignoreThrottle Sets if the throttle should be ignored for this execution
|
||||
*/
|
||||
public void setIgnoreThrottle(boolean ignoreThrottle) {
|
||||
this.ignoreThrottle = ignoreThrottle;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return Should this execution be recorded in the history index
|
||||
*/
|
||||
|
@ -117,34 +98,26 @@ public class ExecuteWatchRequest extends MasterNodeOperationRequest<ExecuteWatch
|
|||
}
|
||||
|
||||
/**
|
||||
* @param triggerType the type of trigger to use
|
||||
* @param triggerSource the trigger source to use
|
||||
*/
|
||||
public void setTriggerEvent(String triggerType, BytesReference triggerSource) {
|
||||
this.triggerType = triggerType;
|
||||
this.triggerSource = triggerSource;
|
||||
}
|
||||
|
||||
/**
|
||||
* @param triggerEvent the trigger event to use
|
||||
* @param data The data that should be associated with the trigger event.
|
||||
* @throws IOException
|
||||
*/
|
||||
public void setTriggerEvent(TriggerEvent triggerEvent) throws IOException {
|
||||
XContentBuilder jsonBuilder = XContentFactory.jsonBuilder();
|
||||
triggerEvent.toXContent(jsonBuilder, ToXContent.EMPTY_PARAMS);
|
||||
setTriggerEvent(triggerEvent.type(), jsonBuilder.bytes());
|
||||
public void setTriggerData(Map<String, Object> data) throws IOException {
|
||||
this.triggerData = data;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return the type of trigger to use
|
||||
* @param event the trigger event to use
|
||||
* @throws IOException
|
||||
*/
|
||||
public String getTriggerType() { return triggerType; }
|
||||
public void setTriggerEvent(TriggerEvent event) throws IOException {
|
||||
setTriggerData(event.data());
|
||||
}
|
||||
|
||||
/**
|
||||
* @return the trigger to use
|
||||
*/
|
||||
public BytesReference getTriggerSource() {
|
||||
return triggerSource;
|
||||
public Map<String, Object> getTriggerData() {
|
||||
return triggerData;
|
||||
}
|
||||
|
||||
|
||||
|
@ -183,9 +156,6 @@ public class ExecuteWatchRequest extends MasterNodeOperationRequest<ExecuteWatch
|
|||
validationException = ValidateActions.addValidationError(error.message(), validationException);
|
||||
}
|
||||
}
|
||||
if (triggerSource == null || triggerType == null) {
|
||||
validationException = ValidateActions.addValidationError("trigger event is missing", validationException);
|
||||
}
|
||||
return validationException;
|
||||
}
|
||||
|
||||
|
@ -194,13 +164,13 @@ public class ExecuteWatchRequest extends MasterNodeOperationRequest<ExecuteWatch
|
|||
super.readFrom(in);
|
||||
id = in.readString();
|
||||
ignoreCondition = in.readBoolean();
|
||||
ignoreThrottle = in.readBoolean();
|
||||
recordExecution = in.readBoolean();
|
||||
if (in.readBoolean()){
|
||||
alternativeInput = in.readMap();
|
||||
}
|
||||
triggerSource = in.readBytesReference();
|
||||
triggerType = in.readString();
|
||||
if (in.readBoolean()) {
|
||||
triggerData = in.readMap();
|
||||
}
|
||||
long actionModesCount = in.readLong();
|
||||
actionModes = new HashMap<>();
|
||||
for (int i = 0; i < actionModesCount; i++) {
|
||||
|
@ -214,14 +184,15 @@ public class ExecuteWatchRequest extends MasterNodeOperationRequest<ExecuteWatch
|
|||
super.writeTo(out);
|
||||
out.writeString(id);
|
||||
out.writeBoolean(ignoreCondition);
|
||||
out.writeBoolean(ignoreThrottle);
|
||||
out.writeBoolean(recordExecution);
|
||||
out.writeBoolean(alternativeInput != null);
|
||||
if (alternativeInput != null) {
|
||||
out.writeMap(alternativeInput);
|
||||
}
|
||||
out.writeBytesReference(triggerSource);
|
||||
out.writeString(triggerType);
|
||||
out.writeBoolean(triggerData != null);
|
||||
if (triggerData != null) {
|
||||
out.writeMap(triggerData);
|
||||
}
|
||||
out.writeLong(actionModes.size());
|
||||
for (Map.Entry<String, ActionExecutionMode> entry : actionModes.entrySet()) {
|
||||
out.writeString(entry.getKey());
|
||||
|
|
|
@ -8,10 +8,9 @@ package org.elasticsearch.watcher.transport.actions.execute;
|
|||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.action.support.master.MasterNodeOperationRequestBuilder;
|
||||
import org.elasticsearch.client.Client;
|
||||
import org.elasticsearch.common.bytes.BytesReference;
|
||||
import org.elasticsearch.watcher.client.WatcherClient;
|
||||
import org.elasticsearch.watcher.trigger.TriggerEvent;
|
||||
import org.elasticsearch.watcher.execution.ActionExecutionMode;
|
||||
import org.elasticsearch.watcher.trigger.TriggerEvent;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Map;
|
||||
|
@ -44,14 +43,6 @@ public class ExecuteWatchRequestBuilder extends MasterNodeOperationRequestBuilde
|
|||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* @param ignoreThrottle Sets if the throttle should be ignored for this execution
|
||||
*/
|
||||
public ExecuteWatchRequestBuilder setIgnoreThrottle(boolean ignoreThrottle) {
|
||||
request.setIgnoreThrottle(ignoreThrottle);
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* @param recordExecution Sets if this execution be recorded in the history index and reflected in the watch
|
||||
*/
|
||||
|
@ -69,11 +60,10 @@ public class ExecuteWatchRequestBuilder extends MasterNodeOperationRequestBuilde
|
|||
}
|
||||
|
||||
/**
|
||||
* @param triggerType the trigger type to use
|
||||
* @param triggerSource the trigger source to use
|
||||
* @param data The data that should be associated with the trigger event
|
||||
*/
|
||||
public ExecuteWatchRequestBuilder setTriggerEvent(String triggerType, BytesReference triggerSource) {
|
||||
request.setTriggerEvent(triggerType, triggerSource);
|
||||
public ExecuteWatchRequestBuilder setTriggerData(Map<String, Object> data) throws IOException {
|
||||
request.setTriggerData(data);
|
||||
return this;
|
||||
}
|
||||
|
||||
|
|
|
@ -84,7 +84,9 @@ public class TransportExecuteWatchAction extends WatcherTransportAction<ExecuteW
|
|||
throw new WatcherException("watch [{}] does not exist", request.getId());
|
||||
}
|
||||
|
||||
TriggerEvent triggerEvent = triggerService.parseTriggerEvent(watch.id(), watch.id() + "_manual_execution", request.getTriggerType(), request.getTriggerSource());
|
||||
String triggerType = watch.trigger().type();
|
||||
TriggerEvent triggerEvent = triggerService.simulateEvent(triggerType, watch.id(), request.getTriggerData());
|
||||
|
||||
ManualExecutionContext.Builder ctxBuilder = ManualExecutionContext.builder(watch, new ManualTriggerEvent(triggerEvent.jobName(), triggerEvent), executionService.defaultThrottlePeriod());
|
||||
|
||||
DateTime executionTime = clock.now(UTC);
|
||||
|
|
|
@ -5,10 +5,12 @@
|
|||
*/
|
||||
package org.elasticsearch.watcher.trigger;
|
||||
|
||||
import org.elasticsearch.common.Nullable;
|
||||
import org.elasticsearch.common.xcontent.XContentParser;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collection;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
*
|
||||
|
@ -37,6 +39,8 @@ public interface TriggerEngine<T extends Trigger, E extends TriggerEvent> {
|
|||
*/
|
||||
boolean remove(String jobId);
|
||||
|
||||
E simulateEvent(String jobId, @Nullable Map<String, Object> data, TriggerService service);
|
||||
|
||||
T parseTrigger(String context, XContentParser parser) throws IOException;
|
||||
|
||||
E parseTriggerEvent(TriggerService service, String watchId, String context, XContentParser parser) throws IOException;
|
||||
|
|
|
@ -16,6 +16,7 @@ import org.elasticsearch.common.xcontent.XContentParser;
|
|||
import java.io.IOException;
|
||||
import java.util.Collection;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.CopyOnWriteArrayList;
|
||||
|
||||
|
@ -74,6 +75,14 @@ public class TriggerService extends AbstractComponent {
|
|||
listeners.add(listener);
|
||||
}
|
||||
|
||||
public TriggerEvent simulateEvent(String type, String jobId, Map<String, Object> data) {
|
||||
TriggerEngine engine = engines.get(type);
|
||||
if (engine == null) {
|
||||
throw new TriggerException("could not simulate trigger event. unknown trigger type [{}]", type);
|
||||
}
|
||||
return engine.simulateEvent(jobId, data, this);
|
||||
}
|
||||
|
||||
public Trigger parseTrigger(String jobName, XContentParser parser) throws IOException {
|
||||
XContentParser.Token token = parser.currentToken();
|
||||
assert token == XContentParser.Token.START_OBJECT;
|
||||
|
|
|
@ -10,8 +10,10 @@ import org.elasticsearch.common.xcontent.XContentParser;
|
|||
import org.elasticsearch.watcher.trigger.TriggerEngine;
|
||||
import org.elasticsearch.watcher.trigger.TriggerService;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.io.IOException;
|
||||
import java.util.Collection;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
*/
|
||||
|
@ -55,6 +57,22 @@ public class ManualTriggerEngine implements TriggerEngine<ManualTrigger,ManualTr
|
|||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ManualTriggerEvent simulateEvent(String jobId, @Nullable Map<String, Object> data, TriggerService service) {
|
||||
if (data == null) {
|
||||
throw new ManualTriggerException("could not simulate manual trigger event. missing required simulated trigger type");
|
||||
}
|
||||
if (data.size() == 1) {
|
||||
String type = data.keySet().iterator().next();
|
||||
return new ManualTriggerEvent(jobId, service.simulateEvent(type, jobId, data));
|
||||
}
|
||||
Object type = data.get("type");
|
||||
if (type instanceof String) {
|
||||
return new ManualTriggerEvent(jobId, service.simulateEvent((String) type, jobId, data));
|
||||
}
|
||||
throw new ManualTriggerException("could not simulate manual trigger event. could not resolve simulated trigger type");
|
||||
}
|
||||
|
||||
@Override
|
||||
public ManualTrigger parseTrigger(String context, XContentParser parser) throws IOException {
|
||||
return ManualTrigger.parse(parser);
|
||||
|
|
|
@ -0,0 +1,22 @@
|
|||
/*
|
||||
* 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.watcher.trigger.manual;
|
||||
|
||||
import org.elasticsearch.watcher.trigger.TriggerException;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
public class ManualTriggerException extends TriggerException {
|
||||
|
||||
public ManualTriggerException(String msg, Object... args) {
|
||||
super(msg, args);
|
||||
}
|
||||
|
||||
public ManualTriggerException(String msg, Throwable cause, Object... args) {
|
||||
super(msg, cause, args);
|
||||
}
|
||||
}
|
|
@ -5,13 +5,17 @@
|
|||
*/
|
||||
package org.elasticsearch.watcher.trigger.schedule;
|
||||
|
||||
import org.elasticsearch.common.joda.time.DateTime;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.xcontent.XContentParser;
|
||||
import org.elasticsearch.watcher.support.WatcherDateTimeUtils;
|
||||
import org.elasticsearch.watcher.support.clock.Clock;
|
||||
import org.elasticsearch.watcher.trigger.AbstractTriggerEngine;
|
||||
import org.elasticsearch.watcher.trigger.TriggerService;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.io.IOException;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
*
|
||||
|
@ -34,6 +38,28 @@ public abstract class ScheduleTriggerEngine extends AbstractTriggerEngine<Schedu
|
|||
return TYPE;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ScheduleTriggerEvent simulateEvent(String jobId, @Nullable Map<String, Object> data, TriggerService service) {
|
||||
DateTime now = clock.nowUTC();
|
||||
if (data == null) {
|
||||
return new ScheduleTriggerEvent(jobId, now, now);
|
||||
}
|
||||
|
||||
Object value = data.get(ScheduleTriggerEvent.Field.TRIGGERED_TIME.getPreferredName());
|
||||
DateTime triggeredTime = value != null ? WatcherDateTimeUtils.convertToDate(value, clock) : now;
|
||||
if (triggeredTime == null) {
|
||||
throw new ScheduleTriggerException("could not simulate schedule event. could not convert provided triggered time [{}] to date/time", value);
|
||||
}
|
||||
|
||||
value = data.get(ScheduleTriggerEvent.Field.SCHEDULED_TIME.getPreferredName());
|
||||
DateTime scheduledTime = value != null ? WatcherDateTimeUtils.convertToDate(value, clock) : triggeredTime;
|
||||
if (scheduledTime == null) {
|
||||
throw new ScheduleTriggerException("could not simulate schedule event. could not convert provided scheduled time [{}] to date/time", value);
|
||||
}
|
||||
|
||||
return new ScheduleTriggerEvent(jobId, triggeredTime, scheduledTime);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ScheduleTrigger parseTrigger(String context, XContentParser parser) throws IOException {
|
||||
Schedule schedule = scheduleRegistry.parse(context, parser);
|
||||
|
|
|
@ -34,6 +34,11 @@ public class ClockMock implements Clock {
|
|||
return now;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DateTime nowUTC() {
|
||||
return now(DateTimeZone.UTC);
|
||||
}
|
||||
|
||||
@Override
|
||||
public DateTime now(DateTimeZone timeZone) {
|
||||
return now.toDateTime(timeZone);
|
||||
|
|
|
@ -50,11 +50,13 @@ import org.elasticsearch.watcher.license.LicenseService;
|
|||
import org.elasticsearch.watcher.support.clock.ClockMock;
|
||||
import org.elasticsearch.watcher.support.http.HttpClient;
|
||||
import org.elasticsearch.watcher.support.init.proxy.ScriptServiceProxy;
|
||||
import org.elasticsearch.watcher.support.xcontent.XContentSource;
|
||||
import org.elasticsearch.watcher.trigger.ScheduleTriggerEngineMock;
|
||||
import org.elasticsearch.watcher.trigger.TriggerService;
|
||||
import org.elasticsearch.watcher.trigger.schedule.ScheduleModule;
|
||||
import org.elasticsearch.watcher.watch.Watch;
|
||||
import org.elasticsearch.watcher.watch.WatchStore;
|
||||
import org.hamcrest.Matcher;
|
||||
import org.junit.After;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Before;
|
||||
|
@ -281,6 +283,10 @@ public abstract class AbstractWatcherIntegrationTests extends ElasticsearchInteg
|
|||
return getInstanceFromMaster(LicenseService.class);
|
||||
}
|
||||
|
||||
protected void assertValue(XContentSource source, String path, Matcher<?> matcher) {
|
||||
assertThat(source.getValue(path), (Matcher<Object>) matcher);
|
||||
}
|
||||
|
||||
protected void assertWatchWithMinimumPerformedActionsCount(final String watchName, final long minimumExpectedWatchActionsWithActionPerformed) throws Exception {
|
||||
assertWatchWithMinimumPerformedActionsCount(watchName, minimumExpectedWatchActionsWithActionPerformed, true);
|
||||
}
|
||||
|
|
|
@ -13,6 +13,7 @@ import org.elasticsearch.common.xcontent.support.XContentMapValues;
|
|||
import org.elasticsearch.watcher.actions.email.service.EmailTemplate;
|
||||
import org.elasticsearch.watcher.actions.email.service.support.EmailServer;
|
||||
import org.elasticsearch.watcher.client.WatcherClient;
|
||||
import org.elasticsearch.watcher.execution.ActionExecutionMode;
|
||||
import org.elasticsearch.watcher.shield.ShieldSecretService;
|
||||
import org.elasticsearch.watcher.support.secret.SecretService;
|
||||
import org.elasticsearch.watcher.support.xcontent.XContentSource;
|
||||
|
@ -138,8 +139,8 @@ public class EmailSecretsIntegrationTests extends AbstractWatcherIntegrationTest
|
|||
TriggerEvent triggerEvent = new ScheduleTriggerEvent(new DateTime(UTC), new DateTime(UTC));
|
||||
ExecuteWatchResponse executeResponse = watcherClient.prepareExecuteWatch("_id")
|
||||
.setRecordExecution(false)
|
||||
.setIgnoreThrottle(true)
|
||||
.setTriggerEvent(triggerEvent)
|
||||
.setActionMode("_all", ActionExecutionMode.FORCE_EXECUTE)
|
||||
.get();
|
||||
assertThat(executeResponse, notNullValue());
|
||||
contentSource = executeResponse.getRecordSource();
|
||||
|
|
|
@ -15,6 +15,7 @@ import org.elasticsearch.common.settings.Settings;
|
|||
import org.elasticsearch.common.xcontent.support.XContentMapValues;
|
||||
import org.elasticsearch.watcher.WatcherException;
|
||||
import org.elasticsearch.watcher.client.WatcherClient;
|
||||
import org.elasticsearch.watcher.execution.ActionExecutionMode;
|
||||
import org.elasticsearch.watcher.shield.ShieldSecretService;
|
||||
import org.elasticsearch.watcher.support.http.HttpRequestTemplate;
|
||||
import org.elasticsearch.watcher.support.http.auth.basic.ApplicableBasicAuth;
|
||||
|
@ -147,8 +148,8 @@ public class HttpSecretsIntegrationTests extends AbstractWatcherIntegrationTests
|
|||
TriggerEvent triggerEvent = new ScheduleTriggerEvent(new DateTime(UTC), new DateTime(UTC));
|
||||
ExecuteWatchResponse executeResponse = watcherClient.prepareExecuteWatch("_id")
|
||||
.setRecordExecution(false)
|
||||
.setIgnoreThrottle(true)
|
||||
.setTriggerEvent(triggerEvent)
|
||||
.setActionMode("_all", ActionExecutionMode.FORCE_EXECUTE)
|
||||
.get();
|
||||
assertThat(executeResponse, notNullValue());
|
||||
contentSource = executeResponse.getRecordSource();
|
||||
|
@ -219,7 +220,7 @@ public class HttpSecretsIntegrationTests extends AbstractWatcherIntegrationTests
|
|||
TriggerEvent triggerEvent = new ScheduleTriggerEvent(new DateTime(UTC), new DateTime(UTC));
|
||||
ExecuteWatchResponse executeResponse = watcherClient.prepareExecuteWatch("_id")
|
||||
.setRecordExecution(false)
|
||||
.setIgnoreThrottle(true)
|
||||
.setActionMode("_all", ActionExecutionMode.FORCE_EXECUTE)
|
||||
.setTriggerEvent(triggerEvent)
|
||||
.get();
|
||||
assertThat(executeResponse, notNullValue());
|
||||
|
|
|
@ -1,38 +0,0 @@
|
|||
/*
|
||||
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||
* or more contributor license agreements. Licensed under the Elastic License;
|
||||
* you may not use this file except in compliance with the Elastic License.
|
||||
*/
|
||||
package org.elasticsearch.watcher.test.integration;
|
||||
|
||||
import org.elasticsearch.action.ActionRequestValidationException;
|
||||
import org.elasticsearch.common.joda.time.DateTime;
|
||||
import org.elasticsearch.common.joda.time.DateTimeZone;
|
||||
import org.elasticsearch.watcher.execution.ActionExecutionMode;
|
||||
import org.elasticsearch.watcher.test.AbstractWatcherIntegrationTests;
|
||||
import org.elasticsearch.watcher.trigger.schedule.ScheduleTriggerEvent;
|
||||
import org.junit.Test;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
public class WatchExecuteTests extends AbstractWatcherIntegrationTests {
|
||||
|
||||
|
||||
@Test(expected = ActionRequestValidationException.class)
|
||||
public void testExecute_InvalidWatchId() throws Exception {
|
||||
DateTime now = DateTime.now(DateTimeZone.UTC);
|
||||
watcherClient().prepareExecuteWatch("id with whitespaces")
|
||||
.setTriggerEvent(new ScheduleTriggerEvent(now, now))
|
||||
.get();
|
||||
}
|
||||
|
||||
@Test(expected = ActionRequestValidationException.class)
|
||||
public void testExecute_InvalidActionId() throws Exception {
|
||||
DateTime now = DateTime.now(DateTimeZone.UTC);
|
||||
watcherClient().prepareExecuteWatch("_id")
|
||||
.setTriggerEvent(new ScheduleTriggerEvent(now, now))
|
||||
.setActionMode("id with whitespaces", randomFrom(ActionExecutionMode.values()))
|
||||
.get();
|
||||
}
|
||||
}
|
|
@ -3,7 +3,7 @@
|
|||
* 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.watcher.test.integration;
|
||||
package org.elasticsearch.watcher.transport.action.ack;
|
||||
|
||||
import com.carrotsearch.randomizedtesting.annotations.Repeat;
|
||||
import org.elasticsearch.action.ActionRequestValidationException;
|
|
@ -0,0 +1,368 @@
|
|||
/*
|
||||
* 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.watcher.transport.action.execute;
|
||||
|
||||
import com.carrotsearch.randomizedtesting.annotations.Repeat;
|
||||
import org.elasticsearch.action.ActionRequestValidationException;
|
||||
import org.elasticsearch.common.collect.ImmutableMap;
|
||||
import org.elasticsearch.common.joda.time.DateTime;
|
||||
import org.elasticsearch.common.joda.time.DateTimeZone;
|
||||
import org.elasticsearch.common.unit.TimeValue;
|
||||
import org.elasticsearch.watcher.actions.ActionStatus;
|
||||
import org.elasticsearch.watcher.client.WatcherClient;
|
||||
import org.elasticsearch.watcher.execution.ActionExecutionMode;
|
||||
import org.elasticsearch.watcher.execution.Wid;
|
||||
import org.elasticsearch.watcher.support.WatcherDateTimeUtils;
|
||||
import org.elasticsearch.watcher.support.xcontent.XContentSource;
|
||||
import org.elasticsearch.watcher.test.AbstractWatcherIntegrationTests;
|
||||
import org.elasticsearch.watcher.transport.actions.ack.AckWatchRequestBuilder;
|
||||
import org.elasticsearch.watcher.transport.actions.ack.AckWatchResponse;
|
||||
import org.elasticsearch.watcher.transport.actions.execute.ExecuteWatchRequestBuilder;
|
||||
import org.elasticsearch.watcher.transport.actions.execute.ExecuteWatchResponse;
|
||||
import org.elasticsearch.watcher.transport.actions.get.GetWatchResponse;
|
||||
import org.elasticsearch.watcher.transport.actions.put.PutWatchResponse;
|
||||
import org.elasticsearch.watcher.trigger.schedule.ScheduleTriggerEvent;
|
||||
import org.elasticsearch.watcher.watch.WatchStatus;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
import static org.elasticsearch.watcher.actions.ActionBuilders.loggingAction;
|
||||
import static org.elasticsearch.watcher.client.WatchSourceBuilders.watchBuilder;
|
||||
import static org.elasticsearch.watcher.condition.ConditionBuilders.alwaysCondition;
|
||||
import static org.elasticsearch.watcher.condition.ConditionBuilders.neverCondition;
|
||||
import static org.elasticsearch.watcher.input.InputBuilders.simpleInput;
|
||||
import static org.elasticsearch.watcher.trigger.TriggerBuilders.schedule;
|
||||
import static org.elasticsearch.watcher.trigger.schedule.Schedules.cron;
|
||||
import static org.elasticsearch.watcher.trigger.schedule.Schedules.interval;
|
||||
import static org.hamcrest.Matchers.is;
|
||||
import static org.hamcrest.Matchers.notNullValue;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
public class WatchExecuteTests extends AbstractWatcherIntegrationTests {
|
||||
|
||||
|
||||
@Test(expected = ActionRequestValidationException.class)
|
||||
public void testExecute_InvalidWatchId() throws Exception {
|
||||
DateTime now = DateTime.now(DateTimeZone.UTC);
|
||||
watcherClient().prepareExecuteWatch("id with whitespaces")
|
||||
.setTriggerEvent(new ScheduleTriggerEvent(now, now))
|
||||
.get();
|
||||
}
|
||||
|
||||
@Test(expected = ActionRequestValidationException.class)
|
||||
public void testExecute_InvalidActionId() throws Exception {
|
||||
DateTime now = DateTime.now(DateTimeZone.UTC);
|
||||
watcherClient().prepareExecuteWatch("_id")
|
||||
.setTriggerEvent(new ScheduleTriggerEvent(now, now))
|
||||
.setActionMode("id with whitespaces", randomFrom(ActionExecutionMode.values()))
|
||||
.get();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testExecute_AllDefaults() throws Exception {
|
||||
WatcherClient watcherClient = watcherClient();
|
||||
|
||||
PutWatchResponse putWatchResponse = watcherClient.preparePutWatch()
|
||||
.setId("_id")
|
||||
.setSource(watchBuilder()
|
||||
.trigger(schedule(cron("0/5 * * * * ? 2099")))
|
||||
.input(simpleInput("foo", "bar"))
|
||||
.condition(alwaysCondition())
|
||||
.addAction("log", loggingAction("_text")))
|
||||
.get();
|
||||
|
||||
assertThat(putWatchResponse.isCreated(), is(true));
|
||||
|
||||
ExecuteWatchResponse response = watcherClient.prepareExecuteWatch("_id").get();
|
||||
assertThat(response, notNullValue());
|
||||
assertThat(response.getRecordId(), notNullValue());
|
||||
Wid wid = new Wid(response.getRecordId());
|
||||
assertThat(wid.watchId(), is("_id"));
|
||||
|
||||
XContentSource record = response.getRecordSource();
|
||||
assertValue(record, "watch_id", is("_id"));
|
||||
assertValue(record, "trigger_event.type", is("manual"));
|
||||
assertValue(record, "trigger_event.triggered_time", notNullValue());
|
||||
String triggeredTime = record.getValue("trigger_event.triggered_time");
|
||||
assertValue(record, "trigger_event.manual.schedule.scheduled_time", is(triggeredTime));
|
||||
assertValue(record, "state", is("executed"));
|
||||
assertValue(record, "input.simple.foo", is("bar"));
|
||||
assertValue(record, "condition.always", notNullValue());
|
||||
assertValue(record, "result.execution_time", notNullValue());
|
||||
assertValue(record, "result.execution_duration", notNullValue());
|
||||
assertValue(record, "result.input.type", is("simple"));
|
||||
assertValue(record, "result.input.payload.foo", is("bar"));
|
||||
assertValue(record, "result.condition.type", is("always"));
|
||||
assertValue(record, "result.condition.met", is(true));
|
||||
assertValue(record, "result.actions.0.id", is("log"));
|
||||
assertValue(record, "result.actions.0.type", is("logging"));
|
||||
assertValue(record, "result.actions.0.status", is("success"));
|
||||
assertValue(record, "result.actions.0.logging.logged_text", is("_text"));
|
||||
}
|
||||
|
||||
@Test @Repeat(iterations = 5)
|
||||
public void testExecute_CustomTriggerData() throws Exception {
|
||||
WatcherClient watcherClient = watcherClient();
|
||||
|
||||
PutWatchResponse putWatchResponse = watcherClient.preparePutWatch()
|
||||
.setId("_id")
|
||||
.setSource(watchBuilder()
|
||||
.trigger(schedule(cron("0/5 * * * * ? 2099")))
|
||||
.input(simpleInput("foo", "bar"))
|
||||
.condition(alwaysCondition())
|
||||
.addAction("log", loggingAction("_text")))
|
||||
.get();
|
||||
|
||||
assertThat(putWatchResponse.isCreated(), is(true));
|
||||
|
||||
DateTime triggeredTime = DateTime.now(DateTimeZone.UTC);
|
||||
DateTime scheduledTime = randomBoolean() ? triggeredTime.minusDays(1) : triggeredTime;
|
||||
|
||||
ExecuteWatchRequestBuilder requestBuilder = watcherClient.prepareExecuteWatch("_id");
|
||||
if (randomBoolean()) {
|
||||
Map<String, Object> data = new HashMap<>();
|
||||
data.put("triggered_time", WatcherDateTimeUtils.formatDate(triggeredTime));
|
||||
if (scheduledTime != triggeredTime) {
|
||||
data.put("scheduled_time", WatcherDateTimeUtils.formatDate(scheduledTime));
|
||||
}
|
||||
requestBuilder.setTriggerData(data);
|
||||
} else {
|
||||
ScheduleTriggerEvent event = new ScheduleTriggerEvent(triggeredTime, scheduledTime);
|
||||
requestBuilder.setTriggerEvent(event);
|
||||
}
|
||||
ExecuteWatchResponse response = requestBuilder.get();
|
||||
|
||||
assertThat(response, notNullValue());
|
||||
assertThat(response.getRecordId(), notNullValue());
|
||||
Wid wid = new Wid(response.getRecordId());
|
||||
assertThat(wid.watchId(), is("_id"));
|
||||
|
||||
XContentSource record = response.getRecordSource();
|
||||
assertValue(record, "watch_id", is("_id"));
|
||||
assertValue(record, "trigger_event.type", is("manual"));
|
||||
assertValue(record, "trigger_event.triggered_time", is(WatcherDateTimeUtils.formatDate(triggeredTime)));
|
||||
assertValue(record, "trigger_event.manual.schedule.scheduled_time", is(WatcherDateTimeUtils.formatDate(scheduledTime)));
|
||||
assertValue(record, "state", is("executed"));
|
||||
assertValue(record, "input.simple.foo", is("bar"));
|
||||
assertValue(record, "condition.always", notNullValue());
|
||||
assertValue(record, "result.execution_time", notNullValue());
|
||||
assertValue(record, "result.execution_duration", notNullValue());
|
||||
assertValue(record, "result.input.type", is("simple"));
|
||||
assertValue(record, "result.input.payload.foo", is("bar"));
|
||||
assertValue(record, "result.condition.type", is("always"));
|
||||
assertValue(record, "result.condition.met", is(true));
|
||||
assertValue(record, "result.actions.0.id", is("log"));
|
||||
assertValue(record, "result.actions.0.type", is("logging"));
|
||||
assertValue(record, "result.actions.0.status", is("success"));
|
||||
assertValue(record, "result.actions.0.logging.logged_text", is("_text"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testExecute_AlternativeInput() throws Exception {
|
||||
WatcherClient watcherClient = watcherClient();
|
||||
|
||||
PutWatchResponse putWatchResponse = watcherClient.preparePutWatch()
|
||||
.setId("_id")
|
||||
.setSource(watchBuilder()
|
||||
.trigger(schedule(cron("0/5 * * * * ? 2099")))
|
||||
.input(simpleInput("foo", "bar"))
|
||||
.condition(alwaysCondition())
|
||||
.addAction("log", loggingAction("_text")))
|
||||
.get();
|
||||
|
||||
assertThat(putWatchResponse.isCreated(), is(true));
|
||||
|
||||
ExecuteWatchResponse response = watcherClient.prepareExecuteWatch("_id")
|
||||
.setAlternativeInput(ImmutableMap.<String, Object>of("foo1", "bar1"))
|
||||
.get();
|
||||
assertThat(response, notNullValue());
|
||||
assertThat(response.getRecordId(), notNullValue());
|
||||
Wid wid = new Wid(response.getRecordId());
|
||||
assertThat(wid.watchId(), is("_id"));
|
||||
|
||||
XContentSource record = response.getRecordSource();
|
||||
assertValue(record, "watch_id", is("_id"));
|
||||
assertValue(record, "trigger_event.type", is("manual"));
|
||||
assertValue(record, "trigger_event.triggered_time", notNullValue());
|
||||
String triggeredTime = record.getValue("trigger_event.triggered_time");
|
||||
assertValue(record, "trigger_event.manual.schedule.scheduled_time", is(triggeredTime));
|
||||
assertValue(record, "state", is("executed"));
|
||||
assertValue(record, "input.simple.foo", is("bar")); // this is the original input
|
||||
assertValue(record, "condition.always", notNullValue());
|
||||
assertValue(record, "result.execution_time", notNullValue());
|
||||
assertValue(record, "result.execution_duration", notNullValue());
|
||||
assertValue(record, "result.input.type", is("simple"));
|
||||
assertValue(record, "result.input.payload.foo1", is("bar1")); // this is the alternative one
|
||||
assertValue(record, "result.condition.type", is("always"));
|
||||
assertValue(record, "result.condition.met", is(true));
|
||||
assertValue(record, "result.actions.0.id", is("log"));
|
||||
assertValue(record, "result.actions.0.type", is("logging"));
|
||||
assertValue(record, "result.actions.0.status", is("success"));
|
||||
assertValue(record, "result.actions.0.logging.logged_text", is("_text"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testExecute_IgnoreCondition() throws Exception {
|
||||
WatcherClient watcherClient = watcherClient();
|
||||
|
||||
PutWatchResponse putWatchResponse = watcherClient.preparePutWatch()
|
||||
.setId("_id")
|
||||
.setSource(watchBuilder()
|
||||
.trigger(schedule(cron("0/5 * * * * ? 2099")))
|
||||
.input(simpleInput("foo", "bar"))
|
||||
.condition(neverCondition())
|
||||
.addAction("log", loggingAction("_text")))
|
||||
.get();
|
||||
|
||||
assertThat(putWatchResponse.isCreated(), is(true));
|
||||
|
||||
ExecuteWatchResponse response = watcherClient.prepareExecuteWatch("_id")
|
||||
.setIgnoreCondition(true)
|
||||
.get();
|
||||
|
||||
assertThat(response, notNullValue());
|
||||
assertThat(response.getRecordId(), notNullValue());
|
||||
Wid wid = new Wid(response.getRecordId());
|
||||
assertThat(wid.watchId(), is("_id"));
|
||||
|
||||
XContentSource record = response.getRecordSource();
|
||||
assertValue(record, "watch_id", is("_id"));
|
||||
assertValue(record, "trigger_event.type", is("manual"));
|
||||
assertValue(record, "trigger_event.triggered_time", notNullValue());
|
||||
String triggeredTime = record.getValue("trigger_event.triggered_time");
|
||||
assertValue(record, "trigger_event.manual.schedule.scheduled_time", is(triggeredTime));
|
||||
assertValue(record, "state", is("executed"));
|
||||
assertValue(record, "input.simple.foo", is("bar"));
|
||||
assertValue(record, "condition.never", notNullValue()); // the original condition
|
||||
assertValue(record, "result.execution_time", notNullValue());
|
||||
assertValue(record, "result.execution_duration", notNullValue());
|
||||
assertValue(record, "result.input.type", is("simple"));
|
||||
assertValue(record, "result.input.payload.foo", is("bar"));
|
||||
assertValue(record, "result.condition.type", is("always")); // when ignored, the condition is replaced with "always"
|
||||
assertValue(record, "result.condition.met", is(true));
|
||||
assertValue(record, "result.actions.0.id", is("log"));
|
||||
assertValue(record, "result.actions.0.type", is("logging"));
|
||||
assertValue(record, "result.actions.0.status", is("success"));
|
||||
assertValue(record, "result.actions.0.logging.logged_text", is("_text"));
|
||||
}
|
||||
|
||||
@Test @Repeat(iterations = 20)
|
||||
public void testExecute_ActionMode() throws Exception {
|
||||
final WatcherClient watcherClient = watcherClient();
|
||||
|
||||
PutWatchResponse putWatchResponse = watcherClient.preparePutWatch()
|
||||
.setId("_id")
|
||||
.setSource(watchBuilder()
|
||||
.trigger(schedule(interval("1s"))) // run every second so we can ack it
|
||||
.input(simpleInput("foo", "bar"))
|
||||
.defaultThrottlePeriod(TimeValue.timeValueMillis(0))
|
||||
.condition(alwaysCondition())
|
||||
.addAction("log", loggingAction("_text")))
|
||||
.get();
|
||||
|
||||
assertThat(putWatchResponse.isCreated(), is(true));
|
||||
|
||||
boolean execute = randomBoolean();
|
||||
boolean force = randomBoolean();
|
||||
ActionExecutionMode mode;
|
||||
if (randomBoolean()) {
|
||||
mode = ActionExecutionMode.SKIP;
|
||||
} else {
|
||||
if (execute && force) {
|
||||
mode = ActionExecutionMode.FORCE_EXECUTE;
|
||||
} else if (execute) {
|
||||
mode = ActionExecutionMode.EXECUTE;
|
||||
} else if (force) {
|
||||
mode = ActionExecutionMode.FORCE_SIMULATE;
|
||||
} else {
|
||||
mode = ActionExecutionMode.SIMULATE;
|
||||
}
|
||||
}
|
||||
|
||||
if (mode.force()) {
|
||||
// since we're forcing, lets ack the action, such that it'd suppoed to be throttled
|
||||
// but forcing will ignore the throttling
|
||||
|
||||
// lets wait for the watch to be ackable
|
||||
if (timeWarped()) {
|
||||
timeWarp().scheduler().trigger("_id");
|
||||
} else {
|
||||
assertBusy(new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
GetWatchResponse getWatchResponse = watcherClient.prepareGetWatch("_id").get();
|
||||
assertValue(getWatchResponse.getSource(), "status.actions.log.ack.state", is("ackable"));
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
String[] actionIds = randomFrom(
|
||||
new String[] { "_all" },
|
||||
new String[] { "log" },
|
||||
new String[] { "foo", "_all" },
|
||||
null
|
||||
);
|
||||
AckWatchRequestBuilder ackWatchRequestBuilder = watcherClient.prepareAckWatch("_id");
|
||||
if (actionIds != null) {
|
||||
ackWatchRequestBuilder.setActionIds(actionIds);
|
||||
}
|
||||
AckWatchResponse ackWatchResponse = ackWatchRequestBuilder.get();
|
||||
assertThat(ackWatchResponse, notNullValue());
|
||||
WatchStatus status = ackWatchResponse.getStatus();
|
||||
assertThat(status, notNullValue());
|
||||
ActionStatus actionStatus = status.actionStatus("log");
|
||||
assertThat(actionStatus, notNullValue());
|
||||
assertThat(actionStatus.ackStatus().state(), is(ActionStatus.AckStatus.State.ACKED));
|
||||
}
|
||||
|
||||
ExecuteWatchResponse response = watcherClient.prepareExecuteWatch("_id")
|
||||
.setActionMode(randomBoolean() ? "log" : "_all", mode)
|
||||
.get();
|
||||
assertThat(response, notNullValue());
|
||||
assertThat(response.getRecordId(), notNullValue());
|
||||
Wid wid = new Wid(response.getRecordId());
|
||||
assertThat(wid.watchId(), is("_id"));
|
||||
|
||||
XContentSource record = response.getRecordSource();
|
||||
assertValue(record, "watch_id", is("_id"));
|
||||
assertValue(record, "trigger_event.type", is("manual"));
|
||||
assertValue(record, "trigger_event.triggered_time", notNullValue());
|
||||
String triggeredTime = record.getValue("trigger_event.triggered_time");
|
||||
assertValue(record, "trigger_event.manual.schedule.scheduled_time", is(triggeredTime));
|
||||
if (mode == ActionExecutionMode.SKIP) {
|
||||
assertValue(record, "state", is("throttled"));
|
||||
} else {
|
||||
assertValue(record, "state", is("executed"));
|
||||
}
|
||||
assertValue(record, "input.simple.foo", is("bar"));
|
||||
assertValue(record, "condition.always", notNullValue());
|
||||
assertValue(record, "result.execution_time", notNullValue());
|
||||
assertValue(record, "result.execution_duration", notNullValue());
|
||||
assertValue(record, "result.input.type", is("simple"));
|
||||
assertValue(record, "result.input.payload.foo", is("bar"));
|
||||
assertValue(record, "result.condition.type", is("always"));
|
||||
assertValue(record, "result.condition.met", is(true));
|
||||
assertValue(record, "result.actions.0.id", is("log"));
|
||||
assertValue(record, "result.actions.0.type", is("logging"));
|
||||
switch (mode) {
|
||||
case SKIP: // the action should be manually skipped/throttled
|
||||
assertValue(record, "result.actions.0.status", is("throttled"));
|
||||
assertValue(record, "result.actions.0.reason", is("manually skipped"));
|
||||
break;
|
||||
default:
|
||||
if (mode.simulate()) {
|
||||
assertValue(record, "result.actions.0.status", is("simulated"));
|
||||
} else {
|
||||
assertValue(record, "result.actions.0.status", is("success"));
|
||||
}
|
||||
assertValue(record, "result.actions.0.logging.logged_text", is("_text"));
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,87 @@
|
|||
/*
|
||||
* 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.watcher.transport.action.execute;
|
||||
|
||||
import org.elasticsearch.common.collect.ImmutableMap;
|
||||
import org.elasticsearch.common.joda.time.DateTime;
|
||||
import org.elasticsearch.watcher.client.WatcherClient;
|
||||
import org.elasticsearch.watcher.execution.Wid;
|
||||
import org.elasticsearch.watcher.support.WatcherDateTimeUtils;
|
||||
import org.elasticsearch.watcher.support.xcontent.XContentSource;
|
||||
import org.elasticsearch.watcher.test.AbstractWatcherIntegrationTests;
|
||||
import org.elasticsearch.watcher.transport.actions.execute.ExecuteWatchResponse;
|
||||
import org.elasticsearch.watcher.transport.actions.put.PutWatchResponse;
|
||||
import org.junit.Test;
|
||||
|
||||
import static org.elasticsearch.watcher.actions.ActionBuilders.loggingAction;
|
||||
import static org.elasticsearch.watcher.client.WatchSourceBuilders.watchBuilder;
|
||||
import static org.elasticsearch.watcher.condition.ConditionBuilders.alwaysCondition;
|
||||
import static org.elasticsearch.watcher.input.InputBuilders.simpleInput;
|
||||
import static org.elasticsearch.watcher.trigger.TriggerBuilders.schedule;
|
||||
import static org.elasticsearch.watcher.trigger.schedule.Schedules.cron;
|
||||
import static org.hamcrest.Matchers.is;
|
||||
import static org.hamcrest.Matchers.notNullValue;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
public class WatchExecuteWithDateMathTests extends AbstractWatcherIntegrationTests {
|
||||
|
||||
@Override
|
||||
protected boolean timeWarped() {
|
||||
return true;
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testExecute_CustomTriggerData() throws Exception {
|
||||
WatcherClient watcherClient = watcherClient();
|
||||
|
||||
PutWatchResponse putWatchResponse = watcherClient.preparePutWatch()
|
||||
.setId("_id")
|
||||
.setSource(watchBuilder()
|
||||
.trigger(schedule(cron("0/5 * * * * ? 2099")))
|
||||
.input(simpleInput("foo", "bar"))
|
||||
.condition(alwaysCondition())
|
||||
.addAction("log", loggingAction("_text")))
|
||||
.get();
|
||||
|
||||
assertThat(putWatchResponse.isCreated(), is(true));
|
||||
|
||||
DateTime triggeredTime = timeWarp().clock().nowUTC();
|
||||
DateTime scheduledTime = triggeredTime.plusMinutes(1);
|
||||
|
||||
ExecuteWatchResponse response = watcherClient.prepareExecuteWatch("_id")
|
||||
.setTriggerData(ImmutableMap.<String, Object>builder()
|
||||
.put("triggered_time", "now")
|
||||
.put("scheduled_time", "now+1m")
|
||||
.build())
|
||||
.get();
|
||||
|
||||
assertThat(response, notNullValue());
|
||||
assertThat(response.getRecordId(), notNullValue());
|
||||
Wid wid = new Wid(response.getRecordId());
|
||||
assertThat(wid.watchId(), is("_id"));
|
||||
|
||||
XContentSource record = response.getRecordSource();
|
||||
assertValue(record, "watch_id", is("_id"));
|
||||
assertValue(record, "trigger_event.type", is("manual"));
|
||||
assertValue(record, "trigger_event.triggered_time", is(WatcherDateTimeUtils.formatDate(triggeredTime)));
|
||||
assertValue(record, "trigger_event.manual.schedule.scheduled_time", is(WatcherDateTimeUtils.formatDate(scheduledTime)));
|
||||
assertValue(record, "state", is("executed"));
|
||||
assertValue(record, "input.simple.foo", is("bar"));
|
||||
assertValue(record, "condition.always", notNullValue());
|
||||
assertValue(record, "result.execution_time", notNullValue());
|
||||
assertValue(record, "result.execution_duration", notNullValue());
|
||||
assertValue(record, "result.input.type", is("simple"));
|
||||
assertValue(record, "result.input.payload.foo", is("bar"));
|
||||
assertValue(record, "result.condition.type", is("always"));
|
||||
assertValue(record, "result.condition.met", is(true));
|
||||
assertValue(record, "result.actions.0.id", is("log"));
|
||||
assertValue(record, "result.actions.0.type", is("logging"));
|
||||
assertValue(record, "result.actions.0.status", is("success"));
|
||||
assertValue(record, "result.actions.0.logging.logged_text", is("_text"));
|
||||
}
|
||||
}
|
Loading…
Reference in New Issue