Watcher: Distributed watch execution (elastic/x-pack-elasticsearch#544)
The distribution of watches now happens on the node which holds the watches index, instead of on the master node. This requires several changes to the current implementation. 1. Running on shards and replicas In order to run watches on the nodes with the watches index on its primaries and replicas. To ensure that watches do not run twice, there is a logic which checks the local shards, runs a murmurhash on the id and runs modulo against the number of shards and replicas, this is the way to find out, if a watch should run local. Reloading happens 2. Several master node actions moved to a HandledTransportAction, as they are basically just aliases for indexing actions, among them the put/delete/get watch actions, the acknowledgement action, the de/activate actions 3. Stats action moved to a broadcast node action, because we potentially have to query every node to get watcher statistics 4. Starting/Stopping watcher now is a master node action, which updates the cluster state and then listeners acts on those. Because of this watches can be running on two systems, if you those have different cluster state versions, until the new watcher state is propagated 5. Watcher is started on all nodes now. With the exception of the ticker schedule engine most classes do not need a lot of resources while running. However they have to run, because of the execute watch API, which can hit any node - it does not make sense to find the right shard for this watch and only then execute (as this also has to work with a watch, that has not been stored before) 6. By using a indexing operation listener, each storing of a watch now parses the watch first and only stores on successful parsing 7. Execute watch API now uses the watcher threadpool for execution 8. Getting the number of watches for the stats now simply queries the different execution engines, how many watches are scheduled, so this is not doing a search anymore There will be follow up commits on this one, mainly to ensure BWC compatibility. Original commit: elastic/x-pack-elasticsearch@0adb46e658
This commit is contained in:
parent
dae762913d
commit
c62f6f8177
|
@ -92,6 +92,7 @@ This is an example of the output:
|
|||
"_id": "my_watch_0-2015-06-02T23:17:55.124Z", <1>
|
||||
"watch_record": { <2>
|
||||
"watch_id": "my_watch",
|
||||
"node": "my_node",
|
||||
"messages": [],
|
||||
"trigger_event": {
|
||||
"type": "manual",
|
||||
|
@ -184,6 +185,7 @@ This is an example of the output:
|
|||
// TESTRESPONSE[s/"last_met_condition": "2015-06-02T23:17:55.124Z"/"last_met_condition": "$body.watch_record._status.last_met_condition"/]
|
||||
// TESTRESPONSE[s/"execution_duration": 12608/"execution_duration": "$body.watch_record.result.execution_duration"/]
|
||||
// TESTRESPONSE[s/"id": "AVSHKzPa9zx62AzUzFXY"/"id": "$body.watch_record.result.actions.0.index.response.id"/]
|
||||
// TESTRESPONSE[s/"node": "my_node"/"node": "$body.watch_record.node"/]
|
||||
<1> The id of the watch record as it would be stored in the `.watcher-history` index.
|
||||
<2> The watch record document as it would be stored in the `.watcher-history` index.
|
||||
<3> The watch execution results.
|
||||
|
|
|
@ -137,10 +137,32 @@ appropriate trigger engine. Watches that have a `schedule` trigger are
|
|||
registered with the `scheduler` trigger engine.
|
||||
|
||||
The scheduler tracks time and triggers watches according to their schedules.
|
||||
The scheduler runs on the master node and is bound to the lifecycle of the
|
||||
{watcher} service. When the {watcher} service is stopped, the scheduler
|
||||
stops with it. Trigger engines use a separate thread pool from the one used
|
||||
to execute watches.
|
||||
On each node, that contains one of the `.watches` shards, a scheduler, that is
|
||||
bound to the watcher lifecycle runs. Even though all primaries and replicas are
|
||||
taken into account, when a watch is triggered, watcher also ensures, that each
|
||||
watch is only triggered on one of those shards. The more replica shards you
|
||||
add, the more distributed the watches can be executed. If you add or remove
|
||||
replicas, all watches need to be reloaded. If a shard is relocated, the
|
||||
primary and all replicas of this particular shard will reload.
|
||||
|
||||
Because the watches are executed on the node, where the watch shards are, you can create
|
||||
dedicated watcher nodes by using shard allocation filtering.
|
||||
|
||||
You could configure nodes with a dedicated `node.attr.watcher: true` property and
|
||||
then configure the `.watches` index like this:
|
||||
|
||||
[source,js]
|
||||
------------------------
|
||||
PUT .watches/_settings
|
||||
{
|
||||
"index.routing.allocation.include": "watcher"
|
||||
}
|
||||
------------------------
|
||||
// CONSOLE
|
||||
// TEST[skip:indexes don't assign]
|
||||
|
||||
When the {watcher} service is stopped, the scheduler stops with it. Trigger
|
||||
engines use a separate thread pool from the one used to execute watches.
|
||||
|
||||
When a watch is triggered, {watcher} queues it up for execution. A `watch_record`
|
||||
document is created and added to the watch history and the watch's status is set
|
||||
|
|
|
@ -88,4 +88,4 @@ public class XDocsClientYamlTestSuiteIT extends ESClientYamlSuiteTestCase {
|
|||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -473,6 +473,7 @@ public class XPackPlugin extends Plugin implements ScriptPlugin, ActionPlugin, I
|
|||
|
||||
public void onIndexModule(IndexModule module) {
|
||||
security.onIndexModule(module);
|
||||
watcher.onIndexModule(module);
|
||||
}
|
||||
|
||||
public static void bindFeatureSet(Binder binder, Class<? extends XPackFeatureSet> featureSet) {
|
||||
|
|
|
@ -88,12 +88,13 @@ public class SchedulerEngine {
|
|||
}
|
||||
|
||||
private final Map<String, ActiveSchedule> schedules = ConcurrentCollections.newConcurrentMap();
|
||||
private ScheduledExecutorService scheduler;
|
||||
private final ScheduledExecutorService scheduler;
|
||||
private final Clock clock;
|
||||
private List<Listener> listeners = new CopyOnWriteArrayList<>();
|
||||
private final List<Listener> listeners = new CopyOnWriteArrayList<>();
|
||||
|
||||
public SchedulerEngine(Clock clock) {
|
||||
this.clock = clock;
|
||||
this.scheduler = Executors.newScheduledThreadPool(1, EsExecutors.daemonThreadFactory("trigger_engine_scheduler"));
|
||||
}
|
||||
|
||||
public void register(Listener listener) {
|
||||
|
@ -101,7 +102,6 @@ public class SchedulerEngine {
|
|||
}
|
||||
|
||||
public void start(Collection<Job> jobs) {
|
||||
this.scheduler = Executors.newScheduledThreadPool(1, EsExecutors.daemonThreadFactory("trigger_engine_scheduler"));
|
||||
jobs.forEach(this::add);
|
||||
}
|
||||
|
||||
|
@ -132,6 +132,13 @@ public class SchedulerEngine {
|
|||
return removedSchedule != null;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return The number of currently active/triggered jobs
|
||||
*/
|
||||
public int jobCount() {
|
||||
return schedules.size();
|
||||
}
|
||||
|
||||
protected void notifyListeners(String name, long triggeredTime, long scheduledTime) {
|
||||
final Event event = new Event(name, triggeredTime, scheduledTime);
|
||||
for (Listener listener : listeners) {
|
||||
|
|
|
@ -8,7 +8,6 @@ package org.elasticsearch.xpack.watcher;
|
|||
import org.apache.logging.log4j.Logger;
|
||||
import org.elasticsearch.action.ActionRequest;
|
||||
import org.elasticsearch.action.ActionResponse;
|
||||
import org.elasticsearch.client.Client;
|
||||
import org.elasticsearch.cluster.NamedDiff;
|
||||
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
|
||||
import org.elasticsearch.cluster.metadata.MetaData;
|
||||
|
@ -31,6 +30,7 @@ import org.elasticsearch.common.settings.SettingsFilter;
|
|||
import org.elasticsearch.common.unit.TimeValue;
|
||||
import org.elasticsearch.common.util.concurrent.EsExecutors;
|
||||
import org.elasticsearch.common.xcontent.NamedXContentRegistry;
|
||||
import org.elasticsearch.index.IndexModule;
|
||||
import org.elasticsearch.license.XPackLicenseState;
|
||||
import org.elasticsearch.plugins.ActionPlugin;
|
||||
import org.elasticsearch.plugins.ScriptPlugin;
|
||||
|
@ -181,7 +181,8 @@ public class Watcher implements ActionPlugin, ScriptPlugin {
|
|||
private static final ScriptContext.Plugin SCRIPT_PLUGIN = new ScriptContext.Plugin("xpack", "watch");
|
||||
public static final ScriptContext SCRIPT_CONTEXT = SCRIPT_PLUGIN::getKey;
|
||||
|
||||
private static final Logger logger = Loggers.getLogger(XPackPlugin.class);
|
||||
private static final Logger logger = Loggers.getLogger(Watcher.class);
|
||||
private WatcherIndexingListener listener;
|
||||
|
||||
public List<NamedWriteableRegistry.Entry> getNamedWriteables() {
|
||||
List<NamedWriteableRegistry.Entry> entries = new ArrayList<>();
|
||||
|
@ -201,15 +202,13 @@ public class Watcher implements ActionPlugin, ScriptPlugin {
|
|||
protected final Settings settings;
|
||||
protected final boolean transportClient;
|
||||
protected final boolean enabled;
|
||||
private final boolean transportClientMode;
|
||||
|
||||
public Watcher(Settings settings) {
|
||||
this.settings = settings;
|
||||
transportClient = "transport".equals(settings.get(Client.CLIENT_TYPE_SETTING_S.getKey()));
|
||||
this.enabled = XPackSettings.WATCHER_ENABLED.get(settings);
|
||||
this.transportClientMode = XPackPlugin.transportClientMode(settings);
|
||||
if (enabled && transportClientMode == false) {
|
||||
validAutoCreateIndex(settings);
|
||||
this.transportClient = XPackPlugin.transportClientMode(settings);
|
||||
if (enabled && transportClient == false) {
|
||||
validAutoCreateIndex(settings, logger);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -293,7 +292,7 @@ public class Watcher implements ActionPlugin, ScriptPlugin {
|
|||
final Watch.Parser watchParser = new Watch.Parser(settings, triggerService, registry, inputRegistry, cryptoService, clock);
|
||||
|
||||
final ExecutionService executionService = new ExecutionService(settings, historyStore, triggeredWatchStore, watchExecutor,
|
||||
clock, threadPool, watchParser, watcherClientProxy);
|
||||
clock, threadPool, watchParser, clusterService, watcherClientProxy);
|
||||
|
||||
final Consumer<Iterable<TriggerEvent>> triggerEngineListener = getTriggerEngineListener(executionService);
|
||||
triggerService.register(triggerEngineListener);
|
||||
|
@ -301,15 +300,19 @@ public class Watcher implements ActionPlugin, ScriptPlugin {
|
|||
final WatcherIndexTemplateRegistry watcherIndexTemplateRegistry = new WatcherIndexTemplateRegistry(settings,
|
||||
clusterService.getClusterSettings(), clusterService, threadPool, internalClient);
|
||||
|
||||
final WatcherService watcherService = new WatcherService(settings, triggerService, executionService,
|
||||
watcherIndexTemplateRegistry, watchParser, watcherClientProxy);
|
||||
WatcherService watcherService = new WatcherService(settings, triggerService, triggeredWatchStore, executionService,
|
||||
watchParser, watcherClientProxy);
|
||||
|
||||
final WatcherLifeCycleService watcherLifeCycleService =
|
||||
new WatcherLifeCycleService(settings, threadPool, clusterService, watcherService);
|
||||
|
||||
listener = new WatcherIndexingListener(settings, watchParser, clock, triggerService);
|
||||
clusterService.addListener(listener);
|
||||
|
||||
return Arrays.asList(registry, watcherClient, inputRegistry, historyStore, triggerService, triggeredWatchParser,
|
||||
watcherLifeCycleService, executionService, triggerEngineListener, watcherService, watchParser,
|
||||
configuredTriggerEngine, triggeredWatchStore, watcherSearchTemplateService, watcherClientProxy);
|
||||
configuredTriggerEngine, triggeredWatchStore, watcherSearchTemplateService, watcherClientProxy,
|
||||
watcherIndexTemplateRegistry);
|
||||
}
|
||||
|
||||
protected TriggerEngine getTriggerEngine(Clock clock, ScheduleRegistry scheduleRegistry) {
|
||||
|
@ -338,7 +341,7 @@ public class Watcher implements ActionPlugin, ScriptPlugin {
|
|||
List<Module> modules = new ArrayList<>();
|
||||
modules.add(b -> {
|
||||
XPackPlugin.bindFeatureSet(b, WatcherFeatureSet.class);
|
||||
if (transportClientMode || enabled == false) {
|
||||
if (transportClient || enabled == false) {
|
||||
b.bind(WatcherService.class).toProvider(Providers.of(null));
|
||||
}
|
||||
});
|
||||
|
@ -358,6 +361,7 @@ public class Watcher implements ActionPlugin, ScriptPlugin {
|
|||
settings.add(INDEX_WATCHER_TEMPLATE_VERSION_SETTING);
|
||||
settings.add(MAX_STOP_TIMEOUT_SETTING);
|
||||
settings.add(ExecutionService.DEFAULT_THROTTLE_PERIOD_SETTING);
|
||||
settings.add(TickerScheduleTriggerEngine.TICKER_INTERVAL_SETTING);
|
||||
settings.add(Setting.intSetting("xpack.watcher.execution.scroll.size", 0, Setting.Property.NodeScope));
|
||||
settings.add(Setting.intSetting("xpack.watcher.watch.scroll.size", 0, Setting.Property.NodeScope));
|
||||
settings.add(ENCRYPT_SENSITIVE_DATA_SETTING);
|
||||
|
@ -369,7 +373,6 @@ public class Watcher implements ActionPlugin, ScriptPlugin {
|
|||
settings.add(Setting.simpleString("xpack.watcher.index.rest.direct_access", Setting.Property.NodeScope));
|
||||
settings.add(Setting.simpleString("xpack.watcher.input.search.default_timeout", Setting.Property.NodeScope));
|
||||
settings.add(Setting.simpleString("xpack.watcher.transform.search.default_timeout", Setting.Property.NodeScope));
|
||||
settings.add(Setting.simpleString("xpack.watcher.trigger.schedule.ticker.tick_interval", Setting.Property.NodeScope));
|
||||
settings.add(Setting.simpleString("xpack.watcher.execution.scroll.timeout", Setting.Property.NodeScope));
|
||||
settings.add(Setting.simpleString("xpack.watcher.start_immediately", Setting.Property.NodeScope));
|
||||
|
||||
|
@ -429,7 +432,21 @@ public class Watcher implements ActionPlugin, ScriptPlugin {
|
|||
return SCRIPT_PLUGIN;
|
||||
}
|
||||
|
||||
static void validAutoCreateIndex(Settings settings) {
|
||||
|
||||
public void onIndexModule(IndexModule module) {
|
||||
if (enabled == false || transportClient) {
|
||||
return;
|
||||
}
|
||||
|
||||
assert listener != null;
|
||||
// for now, we only add this index operation listener to indices starting with .watches
|
||||
// this also means, that aliases pointing to this index have to follow this notation
|
||||
if (module.getIndex().getName().startsWith(Watch.INDEX)) {
|
||||
module.addIndexOperationListener(listener);
|
||||
}
|
||||
}
|
||||
|
||||
static void validAutoCreateIndex(Settings settings, Logger logger) {
|
||||
String value = settings.get("action.auto_create_index");
|
||||
if (value == null) {
|
||||
return;
|
||||
|
|
|
@ -0,0 +1,376 @@
|
|||
/*
|
||||
* 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.watcher;
|
||||
|
||||
import org.elasticsearch.ElasticsearchParseException;
|
||||
import org.elasticsearch.cluster.ClusterChangedEvent;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
import org.elasticsearch.cluster.ClusterStateListener;
|
||||
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
||||
import org.elasticsearch.cluster.routing.AllocationId;
|
||||
import org.elasticsearch.cluster.routing.IndexRoutingTable;
|
||||
import org.elasticsearch.cluster.routing.Murmur3HashFunction;
|
||||
import org.elasticsearch.cluster.routing.RoutingNode;
|
||||
import org.elasticsearch.cluster.routing.ShardRouting;
|
||||
import org.elasticsearch.common.component.AbstractComponent;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.util.set.Sets;
|
||||
import org.elasticsearch.common.xcontent.XContentType;
|
||||
import org.elasticsearch.index.engine.Engine;
|
||||
import org.elasticsearch.index.shard.IndexingOperationListener;
|
||||
import org.elasticsearch.index.shard.ShardId;
|
||||
import org.elasticsearch.xpack.watcher.trigger.TriggerService;
|
||||
import org.elasticsearch.xpack.watcher.watch.Watch;
|
||||
import org.elasticsearch.xpack.watcher.watch.WatchStoreUtils;
|
||||
import org.joda.time.DateTime;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.time.Clock;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static org.elasticsearch.cluster.routing.ShardRoutingState.RELOCATING;
|
||||
import static org.elasticsearch.cluster.routing.ShardRoutingState.STARTED;
|
||||
import static org.joda.time.DateTimeZone.UTC;
|
||||
|
||||
/**
|
||||
* This index listener ensures, that watches that are being indexed are put into the trigger service
|
||||
* Because the condition for this might change based on the shard allocation, this class is also a
|
||||
* cluster state listener
|
||||
*
|
||||
* Whenever a write operation to the current active watch index is made, this listener checks, if
|
||||
* the document should also be added to the local trigger service
|
||||
*
|
||||
*/
|
||||
final class WatcherIndexingListener extends AbstractComponent implements IndexingOperationListener, ClusterStateListener {
|
||||
|
||||
static final Configuration INACTIVE = new Configuration(null, Collections.emptyMap());
|
||||
|
||||
private final Watch.Parser parser;
|
||||
private final Clock clock;
|
||||
private final TriggerService triggerService;
|
||||
private volatile Configuration configuration = INACTIVE;
|
||||
|
||||
WatcherIndexingListener(Settings settings, Watch.Parser parser, Clock clock, TriggerService triggerService) {
|
||||
super(settings);
|
||||
this.parser = parser;
|
||||
this.clock = clock;
|
||||
this.triggerService = triggerService;
|
||||
}
|
||||
|
||||
// package private for testing
|
||||
Configuration getConfiguration() {
|
||||
return configuration;
|
||||
}
|
||||
|
||||
// package private for testing
|
||||
void setConfiguration(Configuration configuration) {
|
||||
this.configuration = configuration;
|
||||
}
|
||||
|
||||
/**
|
||||
* single watch operations that check if the local trigger service should trigger for this
|
||||
* concrete watch
|
||||
*
|
||||
* Watch parsing could be optimized, so that parsing only happens on primary and where the
|
||||
* shard is supposed to be put into the trigger service at some point, right no we dont care
|
||||
*
|
||||
* Note, we have to parse on the primary, because otherwise a failure on the replica when
|
||||
* parsing the watch would result in failing
|
||||
* the replica
|
||||
*
|
||||
* @param shardId The shard id object of the document being processed
|
||||
* @param operation The index operation
|
||||
* @return The index operation
|
||||
*/
|
||||
@Override
|
||||
public Engine.Index preIndex(ShardId shardId, Engine.Index operation) {
|
||||
if (isWatchDocument(shardId.getIndexName(), operation.type())) {
|
||||
DateTime now = new DateTime(clock.millis(), UTC);
|
||||
try {
|
||||
Watch watch = parser.parseWithSecrets(operation.id(), true,
|
||||
operation.source(), now, XContentType.JSON);
|
||||
ShardAllocationConfiguration shardAllocationConfiguration =
|
||||
configuration.localShards.get(shardId);
|
||||
if (shardAllocationConfiguration == null) {
|
||||
logger.debug("no distributed watch execution info found for watch [{}] on shard [{}], got configuration for {}",
|
||||
watch.id(), shardId, configuration.localShards.keySet());
|
||||
return operation;
|
||||
}
|
||||
|
||||
// the watch status is -1, in case a watch has been freshly stored and this save
|
||||
// watch operation does not stem from an execution
|
||||
// we dont need to update the trigger service, when the watch has been updated as
|
||||
// part of an execution, so we can exit early
|
||||
boolean isWatchExecutionOperation = watch.status().version() != -1;
|
||||
if (isWatchExecutionOperation) {
|
||||
return operation;
|
||||
}
|
||||
|
||||
boolean shouldBeTriggered = shardAllocationConfiguration.shouldBeTriggered(watch.id());
|
||||
if (shouldBeTriggered) {
|
||||
if (watch.status().state().isActive()) {
|
||||
triggerService.add(watch);
|
||||
} else {
|
||||
triggerService.remove(watch.id());
|
||||
}
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new ElasticsearchParseException("Could not parse watch with id [{}]", e, operation.id());
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
return operation;
|
||||
}
|
||||
|
||||
/**
|
||||
*
|
||||
* In case of an error, we have to ensure that the triggerservice does not leave anything behind
|
||||
*
|
||||
* TODO: If the configuration changes between preindex and postindex methods and we add a
|
||||
* watch, that could not be indexed
|
||||
* TODO: this watch might not be deleted from the triggerservice. Are we willing to accept this?
|
||||
* TODO: This could be circumvented by using a threadlocal in preIndex(), that contains the
|
||||
* watch and is cleared afterwards
|
||||
*
|
||||
* @param shardId The shard id object of the document being processed
|
||||
* @param index The index operation
|
||||
* @param ex The exception occured during indexing
|
||||
*/
|
||||
@Override
|
||||
public void postIndex(ShardId shardId, Engine.Index index, Exception ex) {
|
||||
if (isWatchDocument(shardId.getIndexName(), index.type())) {
|
||||
triggerService.remove(index.id());
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* If the index operation happened on a watcher shard and is of doc type watcher, we will
|
||||
* remove the watch id from the trigger service
|
||||
*
|
||||
* @param shardId The shard id object of the document being processed
|
||||
* @param delete The delete operation
|
||||
* @return The delete operation
|
||||
*/
|
||||
@Override
|
||||
public Engine.Delete preDelete(ShardId shardId, Engine.Delete delete) {
|
||||
if (isWatchDocument(shardId.getIndexName(), delete.type())) {
|
||||
triggerService.remove(delete.id());
|
||||
}
|
||||
|
||||
return delete;
|
||||
}
|
||||
|
||||
/**
|
||||
* Check if a supplied index and document matches the current configuration for watcher
|
||||
*
|
||||
* @param index The index to check for
|
||||
* @param docType The document type
|
||||
* @return true if this is a watch in the active watcher index, false otherwise
|
||||
*/
|
||||
private boolean isWatchDocument(String index, String docType) {
|
||||
return configuration.isIndexAndActive(index) && docType.equals(Watch.DOC_TYPE);
|
||||
}
|
||||
|
||||
/**
|
||||
* Listen for cluster state changes. This method will start, stop or reload the watcher
|
||||
* service based on cluster state information.
|
||||
* The method checks, if there are local watch indices up and running.
|
||||
*
|
||||
* @param event The ClusterChangedEvent class containing the current and new cluster state
|
||||
*/
|
||||
@Override
|
||||
public void clusterChanged(ClusterChangedEvent event) {
|
||||
if (event.state().nodes().getLocalNode().isDataNode() && event.metaDataChanged()) {
|
||||
try {
|
||||
IndexMetaData metaData = WatchStoreUtils.getConcreteIndex(Watch.INDEX, event.state().metaData());
|
||||
if (metaData == null) {
|
||||
configuration = INACTIVE;
|
||||
} else {
|
||||
checkWatchIndexHasChanged(metaData, event);
|
||||
}
|
||||
} catch (IllegalStateException e) {
|
||||
logger.error("error loading watches index: [{}]", e.getMessage());
|
||||
configuration = INACTIVE;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private void checkWatchIndexHasChanged(IndexMetaData metaData, ClusterChangedEvent event) {
|
||||
String watchIndex = metaData.getIndex().getName();
|
||||
ClusterState state = event.state();
|
||||
String localNodeId = state.getNodes().getLocalNode().getId();
|
||||
RoutingNode routingNode = state.getRoutingNodes().node(localNodeId);
|
||||
|
||||
// no local shards, exit early
|
||||
List<ShardRouting> localShardRouting = routingNode.shardsWithState(watchIndex, STARTED, RELOCATING);
|
||||
if (localShardRouting.isEmpty()) {
|
||||
configuration = INACTIVE;
|
||||
} else {
|
||||
reloadConfiguration(watchIndex, localShardRouting, event);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Reload the configuration if the alias pointing to the watch index was changed or
|
||||
* the index routing table for an index was changed
|
||||
*
|
||||
* @param watchIndex Name of the concrete watches index pointing
|
||||
* @param localShardRouting List of local shards of that index
|
||||
* @param event The cluster changed event containing the new cluster state
|
||||
*/
|
||||
private void reloadConfiguration(String watchIndex, List<ShardRouting> localShardRouting,
|
||||
ClusterChangedEvent event) {
|
||||
// changed alias means to always read a new configuration
|
||||
boolean isAliasChanged = watchIndex.equals(configuration.index) == false;
|
||||
if (isAliasChanged || hasShardAllocationIdChanged(watchIndex, event.state())) {
|
||||
IndexRoutingTable watchIndexRoutingTable = event.state().routingTable().index(watchIndex);
|
||||
Map<ShardId, ShardAllocationConfiguration> ids = getLocalShardAllocationIds(localShardRouting, watchIndexRoutingTable);
|
||||
configuration = new Configuration(watchIndex, ids);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Check if the routing table has changed and local shards are affected
|
||||
*
|
||||
* @param watchIndex Name of the concrete watches index pointing
|
||||
* @param state The new cluster state
|
||||
* @return true if the routing tables has changed and local shards are affected
|
||||
*/
|
||||
private boolean hasShardAllocationIdChanged(String watchIndex, ClusterState state) {
|
||||
List<ShardRouting> allStartedRelocatedShards = state.getRoutingTable().index(watchIndex).shardsWithState(STARTED);
|
||||
allStartedRelocatedShards.addAll(state.getRoutingTable().index(watchIndex).shardsWithState(RELOCATING));
|
||||
|
||||
// exit early, when there are shards, but the current configuration is inactive
|
||||
if (allStartedRelocatedShards.isEmpty() == false && configuration == INACTIVE) {
|
||||
return true;
|
||||
}
|
||||
|
||||
// check for different shard ids
|
||||
String localNodeId = state.nodes().getLocalNodeId();
|
||||
Set<ShardId> clusterStateLocalShardIds = state.getRoutingNodes().node(localNodeId)
|
||||
.shardsWithState(watchIndex, STARTED, RELOCATING).stream()
|
||||
.map(ShardRouting::shardId)
|
||||
.collect(Collectors.toSet());
|
||||
Set<ShardId> configuredLocalShardIds = new HashSet<>(configuration.localShards.keySet());
|
||||
Set<ShardId> differenceSet = Sets.difference(clusterStateLocalShardIds, configuredLocalShardIds);
|
||||
if (differenceSet.isEmpty() == false) {
|
||||
return true;
|
||||
}
|
||||
|
||||
Map<ShardId, List<String>> shards = allStartedRelocatedShards.stream()
|
||||
.collect(Collectors.groupingBy(ShardRouting::shardId,
|
||||
Collectors.mapping(sr -> sr.allocationId().getId(),
|
||||
Collectors.toCollection(ArrayList::new))));
|
||||
|
||||
// sort the collection, so we have a stable order
|
||||
shards.values().forEach(Collections::sort);
|
||||
|
||||
// check for different allocation ids
|
||||
for (Map.Entry<ShardId, ShardAllocationConfiguration> entry : configuration.localShards.entrySet()) {
|
||||
if (shards.containsKey(entry.getKey()) == false) {
|
||||
return true;
|
||||
}
|
||||
|
||||
Collection<String> allocationIds = shards.get(entry.getKey());
|
||||
if (allocationIds.equals(entry.getValue().allocationIds) == false) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
/**
|
||||
* This returns a mapping of the shard it to the index of the shard allocation ids in that
|
||||
* list. The idea here is to have a basis for consistent hashing in order to decide if a
|
||||
* watch needs to be triggered locally or on another system, when it is being indexed
|
||||
* as a single watch action.
|
||||
*
|
||||
* Example:
|
||||
* - ShardId(".watch", 0)
|
||||
* - all allocation ids sorted (in the cluster): [ "a", "b", "c", "d"]
|
||||
* - local allocation id: b (index position 1)
|
||||
* - then store the size of the allocation ids and the index position
|
||||
* data.put(ShardId(".watch", 0), new Tuple(1, 4))
|
||||
*/
|
||||
Map<ShardId, ShardAllocationConfiguration> getLocalShardAllocationIds(List<ShardRouting> localShards, IndexRoutingTable routingTable) {
|
||||
Map<ShardId, ShardAllocationConfiguration> data = new HashMap<>(localShards.size());
|
||||
|
||||
for (ShardRouting shardRouting : localShards) {
|
||||
ShardId shardId = shardRouting.shardId();
|
||||
|
||||
// find all allocation ids for this shard id in the cluster state
|
||||
List<String> allocationIds = routingTable.shard(shardId.getId()).getActiveShards()
|
||||
.stream()
|
||||
.map(ShardRouting::allocationId)
|
||||
.map(AllocationId::getId)
|
||||
.collect(Collectors.toList());
|
||||
|
||||
// sort the list so it is stable
|
||||
Collections.sort(allocationIds);
|
||||
|
||||
String allocationId = shardRouting.allocationId().getId();
|
||||
int idx = allocationIds.indexOf(allocationId);
|
||||
data.put(shardId, new ShardAllocationConfiguration(idx, allocationIds.size(), allocationIds));
|
||||
}
|
||||
|
||||
return data;
|
||||
}
|
||||
|
||||
/**
|
||||
* A helper class, that contains shard configuration per shard id
|
||||
*/
|
||||
static final class Configuration {
|
||||
|
||||
final Map<ShardId, ShardAllocationConfiguration> localShards;
|
||||
final boolean active;
|
||||
final String index;
|
||||
|
||||
Configuration(String index, Map<ShardId, ShardAllocationConfiguration> localShards) {
|
||||
this.active = localShards.isEmpty() == false;
|
||||
this.index = index;
|
||||
this.localShards = Collections.unmodifiableMap(localShards);
|
||||
}
|
||||
|
||||
/**
|
||||
* Find out, if the supplied index matches the current watcher configuration and the
|
||||
* current state is active
|
||||
*
|
||||
* @param index The name of the index to compare with
|
||||
* @return false if watcher is not active or the passed index is not the watcher index
|
||||
*/
|
||||
public boolean isIndexAndActive(String index) {
|
||||
return active == true && index.equals(this.index);
|
||||
}
|
||||
}
|
||||
|
||||
static final class ShardAllocationConfiguration {
|
||||
final int index;
|
||||
final int shardCount;
|
||||
final List<String> allocationIds;
|
||||
|
||||
ShardAllocationConfiguration(int index, int shardCount, List<String> allocationIds) {
|
||||
this.index = index;
|
||||
this.shardCount = shardCount;
|
||||
this.allocationIds = allocationIds;
|
||||
}
|
||||
|
||||
public boolean shouldBeTriggered(String id) {
|
||||
int hash = Murmur3HashFunction.hash(id);
|
||||
int shardIndex = Math.floorMod(hash, shardCount);
|
||||
return shardIndex == index;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -5,48 +5,51 @@
|
|||
*/
|
||||
package org.elasticsearch.xpack.watcher;
|
||||
|
||||
import org.apache.logging.log4j.message.ParameterizedMessage;
|
||||
import org.apache.logging.log4j.util.Supplier;
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.cluster.AckedClusterStateUpdateTask;
|
||||
import org.elasticsearch.cluster.ClusterChangedEvent;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
import org.elasticsearch.cluster.ClusterStateApplier;
|
||||
import org.elasticsearch.cluster.ack.AckedRequest;
|
||||
import org.elasticsearch.cluster.ClusterStateListener;
|
||||
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
||||
import org.elasticsearch.cluster.metadata.MetaData;
|
||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||
import org.elasticsearch.cluster.routing.AllocationId;
|
||||
import org.elasticsearch.cluster.routing.RoutingNode;
|
||||
import org.elasticsearch.cluster.routing.ShardRouting;
|
||||
import org.elasticsearch.cluster.service.ClusterService;
|
||||
import org.elasticsearch.common.component.AbstractComponent;
|
||||
import org.elasticsearch.common.component.LifecycleListener;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.unit.TimeValue;
|
||||
import org.elasticsearch.gateway.GatewayService;
|
||||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
import org.elasticsearch.xpack.watcher.watch.Watch;
|
||||
import org.elasticsearch.xpack.watcher.watch.WatchStoreUtils;
|
||||
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
public class WatcherLifeCycleService extends AbstractComponent implements ClusterStateApplier {
|
||||
import static org.elasticsearch.cluster.routing.ShardRoutingState.RELOCATING;
|
||||
import static org.elasticsearch.cluster.routing.ShardRoutingState.STARTED;
|
||||
|
||||
public class WatcherLifeCycleService extends AbstractComponent implements ClusterStateListener {
|
||||
|
||||
private final ThreadPool threadPool;
|
||||
private final WatcherService watcherService;
|
||||
private final ClusterService clusterService;
|
||||
|
||||
private final ExecutorService executor;
|
||||
private volatile WatcherMetaData watcherMetaData;
|
||||
|
||||
public WatcherLifeCycleService(Settings settings, ThreadPool threadPool, ClusterService clusterService,
|
||||
WatcherService watcherService) {
|
||||
super(settings);
|
||||
this.threadPool = threadPool;
|
||||
this.executor = threadPool.executor(ThreadPool.Names.GENERIC);
|
||||
this.watcherService = watcherService;
|
||||
this.clusterService = clusterService;
|
||||
clusterService.addStateApplier(this);
|
||||
clusterService.addListener(this);
|
||||
// Close if the indices service is being stopped, so we don't run into search failures (locally) that will
|
||||
// happen because we're shutting down and an watch is scheduled.
|
||||
clusterService.addLifecycleListener(new LifecycleListener() {
|
||||
@Override
|
||||
public void beforeStop() {
|
||||
stop(false);
|
||||
stop();
|
||||
}
|
||||
});
|
||||
watcherMetaData = new WatcherMetaData(!settings.getAsBoolean("xpack.watcher.start_immediately", true));
|
||||
|
@ -57,20 +60,7 @@ public class WatcherLifeCycleService extends AbstractComponent implements Cluste
|
|||
}
|
||||
|
||||
public void stop() {
|
||||
stop(true);
|
||||
}
|
||||
|
||||
private synchronized void stop(boolean manual) {
|
||||
WatcherState watcherState = watcherService.state();
|
||||
if (watcherState != WatcherState.STARTED) {
|
||||
logger.debug("not stopping watcher. watcher can only stop if its current state is [{}], but its current state now is [{}]",
|
||||
WatcherState.STARTED, watcherState);
|
||||
} else {
|
||||
watcherService.stop();
|
||||
}
|
||||
if (manual) {
|
||||
updateManualStopped(true);
|
||||
}
|
||||
watcherService.stop();
|
||||
}
|
||||
|
||||
private synchronized void start(ClusterState state, boolean manual) {
|
||||
|
@ -83,7 +73,7 @@ public class WatcherLifeCycleService extends AbstractComponent implements Cluste
|
|||
|
||||
// If we start from a cluster state update we need to check if previously we stopped manually
|
||||
// otherwise Watcher would start upon the next cluster state update while the user instructed Watcher to not run
|
||||
if (!manual && watcherMetaData.manuallyStopped()) {
|
||||
if (!manual && watcherMetaData != null && watcherMetaData.manuallyStopped()) {
|
||||
logger.debug("not starting watcher. watcher was stopped manually and therefore cannot be auto-started");
|
||||
return;
|
||||
}
|
||||
|
@ -98,51 +88,77 @@ public class WatcherLifeCycleService extends AbstractComponent implements Cluste
|
|||
} else {
|
||||
logger.debug("not starting watcher. because the cluster isn't ready yet to run watcher");
|
||||
}
|
||||
if (manual) {
|
||||
updateManualStopped(false);
|
||||
}
|
||||
}
|
||||
|
||||
/*
|
||||
* TODO possible optimization
|
||||
* stop certain parts of watcher, when there are no watcher indices on this node by checking the shardrouting
|
||||
* note that this is not easily possible, because of the execute watch api, that needs to be able to execute anywhere!
|
||||
* this means, only certain components can be stopped
|
||||
*/
|
||||
@Override
|
||||
public void applyClusterState(final ClusterChangedEvent event) {
|
||||
public void clusterChanged(ClusterChangedEvent event) {
|
||||
if (event.state().blocks().hasGlobalBlock(GatewayService.STATE_NOT_RECOVERED_BLOCK)) {
|
||||
// wait until the gateway has recovered from disk, otherwise we think may not have .watches and
|
||||
// a .triggered_watches index, but they may not have been restored from the cluster state on disk
|
||||
return;
|
||||
}
|
||||
|
||||
// find out if watcher was stopped or started manually due to this cluster state change
|
||||
WatcherMetaData watcherMetaData = event.state().getMetaData().custom(WatcherMetaData.TYPE);
|
||||
|
||||
if (watcherMetaData != null) {
|
||||
this.watcherMetaData = watcherMetaData;
|
||||
}
|
||||
|
||||
if (!event.localNodeMaster()) {
|
||||
if (watcherService.state() == WatcherState.STARTED) {
|
||||
// We're no longer the master so we need to stop the watcher.
|
||||
// Stopping the watcher may take a while since it will wait on the scheduler to complete shutdown,
|
||||
// so we fork here so that we don't wait too long. Other events may need to be processed and
|
||||
// other cluster state listeners may need to be executed as well for this event.
|
||||
threadPool.executor(ThreadPool.Names.GENERIC).execute(() -> stop(false));
|
||||
}
|
||||
boolean currentWatcherStopped = watcherMetaData != null && watcherMetaData.manuallyStopped() == true;
|
||||
if (currentWatcherStopped) {
|
||||
logger.debug("watcher manually marked to shutdown in cluster state update, shutting down");
|
||||
executor.execute(this::stop);
|
||||
} else {
|
||||
if (watcherService.state() == WatcherState.STOPPED) {
|
||||
final ClusterState state = event.state();
|
||||
threadPool.executor(ThreadPool.Names.GENERIC).execute(() -> start(state, false));
|
||||
} else {
|
||||
boolean isWatchIndexDeleted = event.indicesDeleted().stream()
|
||||
.filter(index -> Watch.INDEX.equals(index.getName()))
|
||||
.findAny()
|
||||
.isPresent();
|
||||
|
||||
final boolean isWatchIndexOpenInPreviousClusterState = event.previousState().metaData().hasIndex(Watch.INDEX) &&
|
||||
event.previousState().metaData().index(Watch.INDEX).getState() == IndexMetaData.State.OPEN;
|
||||
final boolean isWatchIndexClosedInCurrentClusterState = event.state().metaData().hasIndex(Watch.INDEX) &&
|
||||
event.state().metaData().index(Watch.INDEX).getState() == IndexMetaData.State.CLOSE;
|
||||
final boolean hasWatcherIndexBeenClosed = isWatchIndexOpenInPreviousClusterState && isWatchIndexClosedInCurrentClusterState;
|
||||
|
||||
if (isWatchIndexDeleted || hasWatcherIndexBeenClosed) {
|
||||
threadPool.executor(ThreadPool.Names.GENERIC).execute(() -> watcherService.watchIndexDeletedOrClosed());
|
||||
if (watcherService.state() == WatcherState.STARTED && event.state().nodes().getLocalNode().isDataNode()) {
|
||||
DiscoveryNode localNode = event.state().nodes().getLocalNode();
|
||||
RoutingNode routingNode = event.state().getRoutingNodes().node(localNode.getId());
|
||||
IndexMetaData watcherIndexMetaData = WatchStoreUtils.getConcreteIndex(Watch.INDEX,
|
||||
event.state().metaData());
|
||||
// no watcher index, time to pause
|
||||
if (watcherIndexMetaData == null) {
|
||||
executor.execute(watcherService::pauseExecution);
|
||||
return;
|
||||
}
|
||||
|
||||
String watchIndex = watcherIndexMetaData.getIndex().getName();
|
||||
List<ShardRouting> localShards = routingNode.shardsWithState(watchIndex, RELOCATING, STARTED);
|
||||
|
||||
// no local shards, empty out watcher and not waste resources!
|
||||
if (localShards.isEmpty()) {
|
||||
executor.execute(watcherService::pauseExecution);
|
||||
return;
|
||||
}
|
||||
|
||||
List<String> previousAllocationIds = event.previousState().getRoutingNodes().node(localNode.getId())
|
||||
.shardsWithState(Watch.INDEX, RELOCATING, STARTED).stream()
|
||||
.map(ShardRouting::allocationId)
|
||||
.map(AllocationId::getId)
|
||||
.collect(Collectors.toList());
|
||||
|
||||
List<String> currentAllocationIds = localShards.stream()
|
||||
.map(ShardRouting::allocationId)
|
||||
.map(AllocationId::getId)
|
||||
.collect(Collectors.toList());
|
||||
|
||||
if (previousAllocationIds.size() == currentAllocationIds.size()) {
|
||||
// make sure we can compare the created list of allocation ids
|
||||
Collections.sort(previousAllocationIds);
|
||||
Collections.sort(currentAllocationIds);
|
||||
if (previousAllocationIds.equals(currentAllocationIds) == false) {
|
||||
executor.execute(() -> watcherService.reload(event.state()));
|
||||
}
|
||||
} else {
|
||||
executor.execute(() -> watcherService.reload(event.state()));
|
||||
}
|
||||
} else if (watcherService.state() != WatcherState.STARTED && watcherService.state() != WatcherState.STARTING) {
|
||||
executor.execute(() -> start(event.state(), false));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -151,63 +167,4 @@ public class WatcherLifeCycleService extends AbstractComponent implements Cluste
|
|||
return watcherMetaData;
|
||||
}
|
||||
|
||||
private void updateManualStopped(final boolean stopped) {
|
||||
watcherMetaData = new WatcherMetaData(stopped);
|
||||
|
||||
// We need to make sure that the new WatcherMetaData has arrived on all nodes,
|
||||
// so in order to do this we need to use AckedClusterStateUpdateTask which
|
||||
// requires a AckedRequest and ActionListener...
|
||||
final CountDownLatch latch = new CountDownLatch(1);
|
||||
ActionListener<Boolean> listener = new ActionListener<Boolean>() {
|
||||
@Override
|
||||
public void onResponse(Boolean aBoolean) {
|
||||
latch.countDown();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onFailure(Exception throwable) {
|
||||
logger.warn("updating manually stopped isn't acked", throwable);
|
||||
latch.countDown();
|
||||
}
|
||||
};
|
||||
AckedRequest request = new AckedRequest() {
|
||||
@Override
|
||||
public TimeValue ackTimeout() {
|
||||
return TimeValue.timeValueSeconds(30);
|
||||
}
|
||||
|
||||
@Override
|
||||
public TimeValue masterNodeTimeout() {
|
||||
return TimeValue.timeValueSeconds(30);
|
||||
}
|
||||
};
|
||||
clusterService.submitStateUpdateTask("update_watcher_manually_stopped",
|
||||
new AckedClusterStateUpdateTask<Boolean>(request, listener) {
|
||||
|
||||
@Override
|
||||
protected Boolean newResponse(boolean result) {
|
||||
return result;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ClusterState execute(ClusterState clusterState) throws Exception {
|
||||
ClusterState.Builder builder = new ClusterState.Builder(clusterState);
|
||||
builder.metaData(MetaData.builder(clusterState.getMetaData())
|
||||
.putCustom(WatcherMetaData.TYPE, new WatcherMetaData(stopped)));
|
||||
return builder.build();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onFailure(String source, Exception throwable) {
|
||||
latch.countDown();
|
||||
logger.warn((Supplier<?>) () -> new ParameterizedMessage("couldn't update watcher metadata [{}]", source), throwable);
|
||||
}
|
||||
});
|
||||
try {
|
||||
latch.await();
|
||||
} catch (InterruptedException e) {
|
||||
Thread.currentThread().interrupt();
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -16,6 +16,7 @@ import org.elasticsearch.common.xcontent.XContentParser;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.util.EnumSet;
|
||||
import java.util.Objects;
|
||||
|
||||
public class WatcherMetaData extends AbstractNamedDiffable<MetaData.Custom> implements MetaData.Custom {
|
||||
|
||||
|
@ -54,6 +55,26 @@ public class WatcherMetaData extends AbstractNamedDiffable<MetaData.Custom> impl
|
|||
streamOutput.writeBoolean(manuallyStopped);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "manuallyStopped["+ manuallyStopped +"]";
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (this == o) return true;
|
||||
if (o == null || getClass() != o.getClass()) return false;
|
||||
|
||||
WatcherMetaData action = (WatcherMetaData) o;
|
||||
|
||||
return manuallyStopped == action.manuallyStopped;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Objects.hash(manuallyStopped);
|
||||
}
|
||||
|
||||
public static MetaData.Custom fromXContent(XContentParser parser) throws IOException {
|
||||
XContentParser.Token token;
|
||||
Boolean manuallyStopped = null;
|
||||
|
|
|
@ -15,6 +15,11 @@ import org.elasticsearch.action.search.SearchRequest;
|
|||
import org.elasticsearch.action.search.SearchResponse;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
||||
import org.elasticsearch.cluster.routing.AllocationId;
|
||||
import org.elasticsearch.cluster.routing.Murmur3HashFunction;
|
||||
import org.elasticsearch.cluster.routing.Preference;
|
||||
import org.elasticsearch.cluster.routing.RoutingNode;
|
||||
import org.elasticsearch.cluster.routing.ShardRouting;
|
||||
import org.elasticsearch.common.component.AbstractComponent;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.unit.TimeValue;
|
||||
|
@ -23,7 +28,8 @@ import org.elasticsearch.search.SearchHit;
|
|||
import org.elasticsearch.search.builder.SearchSourceBuilder;
|
||||
import org.elasticsearch.search.sort.SortBuilders;
|
||||
import org.elasticsearch.xpack.watcher.execution.ExecutionService;
|
||||
import org.elasticsearch.xpack.watcher.support.WatcherIndexTemplateRegistry;
|
||||
import org.elasticsearch.xpack.watcher.execution.TriggeredWatch;
|
||||
import org.elasticsearch.xpack.watcher.execution.TriggeredWatchStore;
|
||||
import org.elasticsearch.xpack.watcher.support.init.proxy.WatcherClientProxy;
|
||||
import org.elasticsearch.xpack.watcher.trigger.TriggerService;
|
||||
import org.elasticsearch.xpack.watcher.watch.Watch;
|
||||
|
@ -32,10 +38,16 @@ import org.elasticsearch.xpack.watcher.watch.WatchStoreUtils;
|
|||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
import java.util.Optional;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static org.elasticsearch.cluster.routing.ShardRoutingState.RELOCATING;
|
||||
import static org.elasticsearch.cluster.routing.ShardRoutingState.STARTED;
|
||||
import static org.elasticsearch.xpack.watcher.support.Exceptions.illegalState;
|
||||
import static org.elasticsearch.xpack.watcher.watch.Watch.DOC_TYPE;
|
||||
import static org.elasticsearch.xpack.watcher.watch.Watch.INDEX;
|
||||
|
@ -44,34 +56,74 @@ import static org.elasticsearch.xpack.watcher.watch.Watch.INDEX;
|
|||
public class WatcherService extends AbstractComponent {
|
||||
|
||||
private final TriggerService triggerService;
|
||||
private final TriggeredWatchStore triggeredWatchStore;
|
||||
private final ExecutionService executionService;
|
||||
private final WatcherIndexTemplateRegistry watcherIndexTemplateRegistry;
|
||||
// package-private for testing
|
||||
final AtomicReference<WatcherState> state = new AtomicReference<>(WatcherState.STOPPED);
|
||||
private final TimeValue scrollTimeout;
|
||||
private final int scrollSize;
|
||||
private final Watch.Parser parser;
|
||||
private final WatcherClientProxy client;
|
||||
// package-private for testing
|
||||
final AtomicReference<WatcherState> state = new AtomicReference<>(WatcherState.STOPPED);
|
||||
|
||||
public WatcherService(Settings settings, TriggerService triggerService, ExecutionService executionService,
|
||||
WatcherIndexTemplateRegistry watcherIndexTemplateRegistry, Watch.Parser parser, WatcherClientProxy client) {
|
||||
public WatcherService(Settings settings, TriggerService triggerService, TriggeredWatchStore triggeredWatchStore,
|
||||
ExecutionService executionService, Watch.Parser parser, WatcherClientProxy client) {
|
||||
super(settings);
|
||||
this.triggerService = triggerService;
|
||||
this.triggeredWatchStore = triggeredWatchStore;
|
||||
this.executionService = executionService;
|
||||
this.watcherIndexTemplateRegistry = watcherIndexTemplateRegistry;
|
||||
this.scrollTimeout = settings.getAsTime("xpack.watcher.watch.scroll.timeout", TimeValue.timeValueSeconds(30));
|
||||
this.scrollSize = settings.getAsInt("xpack.watcher.watch.scroll.size", 100);
|
||||
this.parser = parser;
|
||||
this.client = client;
|
||||
}
|
||||
|
||||
/**
|
||||
* Ensure that watcher can be started, by checking if all indices are marked as up and ready in the cluster state
|
||||
* @param state The current cluster state
|
||||
* @return true if everything is good to go, so that the service can be started
|
||||
*/
|
||||
public boolean validate(ClusterState state) {
|
||||
boolean executionServiceValid = executionService.validate(state);
|
||||
if (executionServiceValid) {
|
||||
try {
|
||||
IndexMetaData indexMetaData = WatchStoreUtils.getConcreteIndex(Watch.INDEX, state.metaData());
|
||||
// no watch index yet means we are good to go
|
||||
if (indexMetaData == null) {
|
||||
return true;
|
||||
} else {
|
||||
if (indexMetaData.getState() == IndexMetaData.State.CLOSE) {
|
||||
logger.debug("watch index [{}] is marked as closed, watcher cannot be started", indexMetaData.getIndex().getName());
|
||||
return false;
|
||||
} else {
|
||||
return state.routingTable().index(indexMetaData.getIndex()).allPrimaryShardsActive();
|
||||
}
|
||||
}
|
||||
} catch (IllegalStateException e) {
|
||||
logger.trace((Supplier<?>) () -> new ParameterizedMessage("error getting index meta data [{}]: ", Watch.INDEX), e);
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
public void start(ClusterState clusterState) throws Exception {
|
||||
// starting already triggered, exit early
|
||||
WatcherState currentState = state.get();
|
||||
if (currentState == WatcherState.STARTING || currentState == WatcherState.STARTED) {
|
||||
throw new IllegalStateException("watcher is already in state ["+ currentState +"]");
|
||||
}
|
||||
|
||||
if (state.compareAndSet(WatcherState.STOPPED, WatcherState.STARTING)) {
|
||||
try {
|
||||
logger.debug("starting watch service...");
|
||||
watcherIndexTemplateRegistry.addTemplatesIfMissing();
|
||||
|
||||
executionService.start(clusterState);
|
||||
triggerService.start(loadWatches(clusterState));
|
||||
Collection<Watch> watches = loadWatches(clusterState);
|
||||
triggerService.start(watches);
|
||||
|
||||
Collection<TriggeredWatch> triggeredWatches = triggeredWatchStore.findTriggeredWatches(watches);
|
||||
executionService.executeTriggeredWatches(triggeredWatches);
|
||||
|
||||
state.set(WatcherState.STARTED);
|
||||
logger.debug("watch service has started");
|
||||
|
@ -79,36 +131,65 @@ public class WatcherService extends AbstractComponent {
|
|||
state.set(WatcherState.STOPPED);
|
||||
throw e;
|
||||
}
|
||||
} else {
|
||||
logger.debug("not starting watcher, because its state is [{}] while [{}] is expected", state, WatcherState.STOPPED);
|
||||
}
|
||||
}
|
||||
|
||||
public boolean validate(ClusterState state) {
|
||||
return executionService.validate(state);
|
||||
}
|
||||
|
||||
/**
|
||||
* Stops the watcher service and it's subservices. Should only be called, when watcher is stopped manually
|
||||
*/
|
||||
public void stop() {
|
||||
if (state.compareAndSet(WatcherState.STARTED, WatcherState.STOPPING)) {
|
||||
logger.debug("stopping watch service...");
|
||||
triggerService.stop();
|
||||
executionService.stop();
|
||||
state.set(WatcherState.STOPPED);
|
||||
logger.debug("watch service has stopped");
|
||||
WatcherState currentState = state.get();
|
||||
if (currentState == WatcherState.STOPPING || currentState == WatcherState.STOPPED) {
|
||||
logger.trace("watcher is already in state [{}] not stopping", currentState);
|
||||
} else {
|
||||
logger.debug("not stopping watcher, because its state is [{}] while [{}] is expected", state, WatcherState.STARTED);
|
||||
try {
|
||||
if (state.compareAndSet(WatcherState.STARTED, WatcherState.STOPPING)) {
|
||||
logger.debug("stopping watch service...");
|
||||
triggerService.stop();
|
||||
executionService.stop();
|
||||
state.set(WatcherState.STOPPED);
|
||||
logger.debug("watch service has stopped");
|
||||
}
|
||||
} catch (Exception e) {
|
||||
state.set(WatcherState.STOPPED);
|
||||
logger.error("Error stopping watcher", e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Reload the watcher service, does not switch the state from stopped to started, just keep going
|
||||
* @param clusterState cluster state, which is needed to find out about local shards
|
||||
*/
|
||||
public void reload(ClusterState clusterState) {
|
||||
pauseExecution();
|
||||
|
||||
// load watches
|
||||
Collection<Watch> watches = loadWatches(clusterState);
|
||||
watches.forEach(triggerService::add);
|
||||
|
||||
// then load triggered watches, which might have been in the queue that we just cleared,
|
||||
// maybe we dont need to execute those anymore however, i.e. due to shard shuffling
|
||||
// then someone else will
|
||||
Collection<TriggeredWatch> triggeredWatches = triggeredWatchStore.findTriggeredWatches(watches);
|
||||
executionService.executeTriggeredWatches(triggeredWatches);
|
||||
}
|
||||
|
||||
/**
|
||||
* Stop execution of watches on this node, do not try to reload anything, but still allow
|
||||
* manual watch execution, i.e. via the execute watch API
|
||||
*/
|
||||
public void pauseExecution() {
|
||||
executionService.pauseExecution();
|
||||
triggerService.pauseExecution();
|
||||
}
|
||||
|
||||
/**
|
||||
* This reads all watches from the .watches index/alias and puts them into memory for a short period of time,
|
||||
* before they are fed into the trigger service.
|
||||
*
|
||||
* This is only invoked when a node becomes master, so either on start up or when a master node switches - while watcher is started up
|
||||
*/
|
||||
private Collection<Watch> loadWatches(ClusterState clusterState) {
|
||||
IndexMetaData indexMetaData = WatchStoreUtils.getConcreteIndex(INDEX, clusterState.metaData());
|
||||
|
||||
// no index exists, all good, we can start
|
||||
if (indexMetaData == null) {
|
||||
return Collections.emptyList();
|
||||
|
@ -119,23 +200,69 @@ public class WatcherService extends AbstractComponent {
|
|||
throw illegalState("not all required shards have been refreshed");
|
||||
}
|
||||
|
||||
// find out local shards
|
||||
String watchIndexName = indexMetaData.getIndex().getName();
|
||||
RoutingNode routingNode = clusterState.getRoutingNodes().node(clusterState.nodes().getLocalNodeId());
|
||||
// yes, this can happen, if the state is not recovered
|
||||
if (routingNode == null) {
|
||||
return Collections.emptyList();
|
||||
}
|
||||
List<ShardRouting> localShards = routingNode.shardsWithState(watchIndexName, RELOCATING, STARTED);
|
||||
|
||||
// find out all allocation ids
|
||||
List<ShardRouting> watchIndexShardRoutings = clusterState.getRoutingTable().allShards(watchIndexName);
|
||||
|
||||
List<Watch> watches = new ArrayList<>();
|
||||
|
||||
SearchRequest searchRequest = new SearchRequest(INDEX)
|
||||
.types(DOC_TYPE)
|
||||
.scroll(scrollTimeout)
|
||||
.preference(Preference.ONLY_LOCAL.toString())
|
||||
.source(new SearchSourceBuilder()
|
||||
.size(scrollSize)
|
||||
.sort(SortBuilders.fieldSort("_doc"))
|
||||
.version(true));
|
||||
SearchResponse response = client.search(searchRequest, null);
|
||||
SearchResponse response = client.search(searchRequest);
|
||||
try {
|
||||
if (response.getTotalShards() != response.getSuccessfulShards()) {
|
||||
throw new ElasticsearchException("Partial response while loading watches");
|
||||
}
|
||||
|
||||
if (response.getHits().getTotalHits() == 0) {
|
||||
return Collections.emptyList();
|
||||
}
|
||||
|
||||
Map<Integer, List<String>> sortedShards = new HashMap<>(localShards.size());
|
||||
for (ShardRouting localShardRouting : localShards) {
|
||||
List<String> sortedAllocationIds = watchIndexShardRoutings.stream()
|
||||
.filter(sr -> localShardRouting.getId() == sr.getId())
|
||||
.map(ShardRouting::allocationId).filter(Objects::nonNull)
|
||||
.map(AllocationId::getId).filter(Objects::nonNull)
|
||||
.sorted()
|
||||
.collect(Collectors.toList());
|
||||
|
||||
sortedShards.put(localShardRouting.getId(), sortedAllocationIds);
|
||||
}
|
||||
|
||||
while (response.getHits().getHits().length != 0) {
|
||||
for (SearchHit hit : response.getHits()) {
|
||||
// find out if this hit should be processed locally
|
||||
Optional<ShardRouting> correspondingShardOptional = localShards.stream()
|
||||
.filter(sr -> sr.shardId().equals(hit.getShard().getShardId()))
|
||||
.findFirst();
|
||||
if (correspondingShardOptional.isPresent() == false) {
|
||||
continue;
|
||||
}
|
||||
ShardRouting correspondingShard = correspondingShardOptional.get();
|
||||
List<String> shardAllocationIds = sortedShards.get(hit.getShard().getShardId().id());
|
||||
// based on the shard allocation ids, get the bucket of the shard, this hit was in
|
||||
int bucket = shardAllocationIds.indexOf(correspondingShard.allocationId().getId());
|
||||
String id = hit.getId();
|
||||
|
||||
if (parseWatchOnThisNode(hit.getId(), shardAllocationIds.size(), bucket) == false) {
|
||||
continue;
|
||||
}
|
||||
|
||||
try {
|
||||
Watch watch = parser.parse(id, true, hit.getSourceRef(), XContentType.JSON);
|
||||
watch.version(hit.getVersion());
|
||||
|
@ -149,10 +276,25 @@ public class WatcherService extends AbstractComponent {
|
|||
} finally {
|
||||
client.clearScroll(response.getScrollId());
|
||||
}
|
||||
|
||||
logger.debug("Loaded [{}] watches for execution", watches.size());
|
||||
|
||||
return watches;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Find out if the watch with this id, should be parsed and triggered on this node
|
||||
*
|
||||
* @param id The id of the watch
|
||||
* @param totalShardCount The count of all primary shards of the current watches index
|
||||
* @param index The index of the local shard
|
||||
* @return true if the we should parse the watch on this node, false otherwise
|
||||
*/
|
||||
private boolean parseWatchOnThisNode(String id, int totalShardCount, int index) {
|
||||
int hash = Murmur3HashFunction.hash(id);
|
||||
int shardIndex = Math.floorMod(hash, totalShardCount);
|
||||
return shardIndex == index;
|
||||
}
|
||||
|
||||
public WatcherState state() {
|
||||
return state.get();
|
||||
|
@ -162,12 +304,4 @@ public class WatcherService extends AbstractComponent {
|
|||
Map<String, Object> innerMap = executionService.usageStats();
|
||||
return innerMap;
|
||||
}
|
||||
|
||||
/**
|
||||
* Something deleted or closed the {@link Watch#INDEX} and thus we need to do some cleanup to prevent further execution of watches
|
||||
* as those watches cannot be updated anymore
|
||||
*/
|
||||
public void watchIndexDeletedOrClosed() {
|
||||
executionService.clearExecutions();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -10,7 +10,7 @@ import org.apache.logging.log4j.util.Supplier;
|
|||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.action.get.GetResponse;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
||||
import org.elasticsearch.cluster.service.ClusterService;
|
||||
import org.elasticsearch.common.component.AbstractComponent;
|
||||
import org.elasticsearch.common.metrics.MeanMetric;
|
||||
import org.elasticsearch.common.settings.Setting;
|
||||
|
@ -31,7 +31,6 @@ import org.elasticsearch.xpack.watcher.support.init.proxy.WatcherClientProxy;
|
|||
import org.elasticsearch.xpack.watcher.transform.Transform;
|
||||
import org.elasticsearch.xpack.watcher.trigger.TriggerEvent;
|
||||
import org.elasticsearch.xpack.watcher.watch.Watch;
|
||||
import org.elasticsearch.xpack.watcher.watch.WatchStoreUtils;
|
||||
import org.joda.time.DateTime;
|
||||
|
||||
import java.io.IOException;
|
||||
|
@ -48,7 +47,7 @@ import java.util.concurrent.atomic.AtomicBoolean;
|
|||
|
||||
import static org.joda.time.DateTimeZone.UTC;
|
||||
|
||||
public final class ExecutionService extends AbstractComponent {
|
||||
public class ExecutionService extends AbstractComponent {
|
||||
|
||||
public static final Setting<TimeValue> DEFAULT_THROTTLE_PERIOD_SETTING =
|
||||
Setting.positiveTimeSetting("xpack.watcher.execution.default_throttle_period",
|
||||
|
@ -65,13 +64,14 @@ public final class ExecutionService extends AbstractComponent {
|
|||
private final TimeValue maxStopTimeout;
|
||||
private final ThreadPool threadPool;
|
||||
private final Watch.Parser parser;
|
||||
private final ClusterService clusterService;
|
||||
private final WatcherClientProxy client;
|
||||
|
||||
private volatile CurrentExecutions currentExecutions;
|
||||
private final AtomicBoolean started = new AtomicBoolean(false);
|
||||
|
||||
public ExecutionService(Settings settings, HistoryStore historyStore, TriggeredWatchStore triggeredWatchStore, WatchExecutor executor,
|
||||
Clock clock, ThreadPool threadPool, Watch.Parser parser,
|
||||
Clock clock, ThreadPool threadPool, Watch.Parser parser, ClusterService clusterService,
|
||||
WatcherClientProxy client) {
|
||||
super(settings);
|
||||
this.historyStore = historyStore;
|
||||
|
@ -82,6 +82,7 @@ public final class ExecutionService extends AbstractComponent {
|
|||
this.maxStopTimeout = Watcher.MAX_STOP_TIMEOUT_SETTING.get(settings);
|
||||
this.threadPool = threadPool;
|
||||
this.parser = parser;
|
||||
this.clusterService = clusterService;
|
||||
this.client = client;
|
||||
}
|
||||
|
||||
|
@ -97,8 +98,6 @@ public final class ExecutionService extends AbstractComponent {
|
|||
historyStore.start();
|
||||
triggeredWatchStore.start();
|
||||
currentExecutions = new CurrentExecutions();
|
||||
Collection<TriggeredWatch> triggeredWatches = triggeredWatchStore.loadTriggeredWatches(state);
|
||||
executeTriggeredWatches(triggeredWatches);
|
||||
logger.debug("started execution service");
|
||||
} catch (Exception e) {
|
||||
started.set(false);
|
||||
|
@ -108,29 +107,7 @@ public final class ExecutionService extends AbstractComponent {
|
|||
}
|
||||
|
||||
public boolean validate(ClusterState state) {
|
||||
boolean triggeredWatchStoreReady = triggeredWatchStore.validate(state);
|
||||
if (triggeredWatchStoreReady == false) {
|
||||
return false;
|
||||
}
|
||||
|
||||
try {
|
||||
IndexMetaData indexMetaData = WatchStoreUtils.getConcreteIndex(Watch.INDEX, state.metaData());
|
||||
// no watch index yet means we are good to go
|
||||
if (indexMetaData == null) {
|
||||
return true;
|
||||
} else {
|
||||
if (indexMetaData.getState() == IndexMetaData.State.CLOSE) {
|
||||
logger.debug("watch index [{}] is marked as closed, watcher cannot be started",
|
||||
indexMetaData.getIndex().getName());
|
||||
return false;
|
||||
} else {
|
||||
return state.routingTable().index(indexMetaData.getIndex()).allPrimaryShardsActive();
|
||||
}
|
||||
}
|
||||
} catch (IllegalStateException e) {
|
||||
logger.trace((Supplier<?>) () -> new ParameterizedMessage("error getting index meta data [{}]: ", Watch.INDEX), e);
|
||||
return false;
|
||||
}
|
||||
return triggeredWatchStore.validate(state) && HistoryStore.validate(state);
|
||||
}
|
||||
|
||||
public void stop() {
|
||||
|
@ -140,16 +117,17 @@ public final class ExecutionService extends AbstractComponent {
|
|||
// this is a forceful shutdown that also interrupts the worker threads in the thread pool
|
||||
int cancelledTaskCount = executor.queue().drainTo(new ArrayList<>());
|
||||
|
||||
currentExecutions.sealAndAwaitEmpty(maxStopTimeout);
|
||||
this.clearExecutions();
|
||||
triggeredWatchStore.stop();
|
||||
historyStore.stop();
|
||||
logger.debug("cancelled [{}] queued tasks", cancelledTaskCount);
|
||||
logger.debug("stopped execution service");
|
||||
logger.debug("stopped execution service, cancelled [{}] queued tasks", cancelledTaskCount);
|
||||
}
|
||||
}
|
||||
|
||||
public boolean started() {
|
||||
return started.get();
|
||||
public synchronized void pauseExecution() {
|
||||
int cancelledTaskCount = executor.queue().drainTo(new ArrayList<>());
|
||||
this.clearExecutions();
|
||||
logger.debug("paused execution service, cancelled [{}] queued tasks", cancelledTaskCount);
|
||||
}
|
||||
|
||||
public TimeValue defaultThrottlePeriod() {
|
||||
|
@ -225,7 +203,7 @@ public final class ExecutionService extends AbstractComponent {
|
|||
}
|
||||
|
||||
if (triggeredWatches.isEmpty() == false) {
|
||||
logger.debug("saving triggered [{}] watches", triggeredWatches.size());
|
||||
logger.trace("saving triggered [{}] watches", triggeredWatches.size());
|
||||
|
||||
triggeredWatchStore.putAll(triggeredWatches, ActionListener.wrap(
|
||||
(slots) -> {
|
||||
|
@ -272,12 +250,16 @@ public final class ExecutionService extends AbstractComponent {
|
|||
}
|
||||
|
||||
public WatchRecord execute(WatchExecutionContext ctx) {
|
||||
ctx.setNodeId(clusterService.localNode().getId());
|
||||
WatchRecord record = null;
|
||||
try {
|
||||
boolean executionAlreadyExists = currentExecutions.put(ctx.watch().id(), new WatchExecution(ctx, Thread.currentThread()));
|
||||
if (executionAlreadyExists) {
|
||||
logger.trace("not executing watch [{}] because it is already queued", ctx.watch().id());
|
||||
record = ctx.abortBeforeExecution(ExecutionState.NOT_EXECUTED_ALREADY_QUEUED, "Watch is already queued in thread pool");
|
||||
} else if (ctx.watch().status().state().isActive() == false) {
|
||||
logger.debug("not executing watch [{}] because it is marked as inactive", ctx.watch().id());
|
||||
record = ctx.abortBeforeExecution(ExecutionState.EXECUTION_NOT_NEEDED, "Watch is not active");
|
||||
} else {
|
||||
boolean watchExists = false;
|
||||
try {
|
||||
|
@ -342,9 +324,9 @@ public final class ExecutionService extends AbstractComponent {
|
|||
private void logWatchRecord(WatchExecutionContext ctx, Exception e) {
|
||||
// failed watches stack traces are only logged in debug, otherwise they should be checked out in the history
|
||||
if (logger.isDebugEnabled()) {
|
||||
logger.debug((Supplier<?>) () -> new ParameterizedMessage("failed to execute watch [{}]", ctx.id()), e);
|
||||
logger.debug((Supplier<?>) () -> new ParameterizedMessage("failed to execute watch [{}]", ctx.watch().id()), e);
|
||||
} else {
|
||||
logger.warn("Failed to execute watch [{}]", ctx.id());
|
||||
logger.warn("Failed to execute watch [{}]", ctx.watch().id());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -439,7 +421,7 @@ public final class ExecutionService extends AbstractComponent {
|
|||
return record;
|
||||
}
|
||||
|
||||
void executeTriggeredWatches(Collection<TriggeredWatch> triggeredWatches) throws Exception {
|
||||
public void executeTriggeredWatches(Collection<TriggeredWatch> triggeredWatches) {
|
||||
assert triggeredWatches != null;
|
||||
int counter = 0;
|
||||
for (TriggeredWatch triggeredWatch : triggeredWatches) {
|
||||
|
@ -448,16 +430,20 @@ public final class ExecutionService extends AbstractComponent {
|
|||
String message = "unable to find watch for record [" + triggeredWatch.id().watchId() + "]/[" + triggeredWatch.id() +
|
||||
"], perhaps it has been deleted, ignoring...";
|
||||
WatchRecord record = new WatchRecord.MessageWatchRecord(triggeredWatch.id(), triggeredWatch.triggerEvent(),
|
||||
ExecutionState.NOT_EXECUTED_WATCH_MISSING, message);
|
||||
ExecutionState.NOT_EXECUTED_WATCH_MISSING, message, clusterService.localNode().getId());
|
||||
historyStore.forcePut(record);
|
||||
triggeredWatchStore.delete(triggeredWatch.id());
|
||||
} else {
|
||||
DateTime now = new DateTime(clock.millis(), UTC);
|
||||
Watch watch = parser.parseWithSecrets(response.getId(), true, response.getSourceAsBytesRef(), now, XContentType.JSON);
|
||||
TriggeredExecutionContext ctx =
|
||||
new StartupExecutionContext(watch, now, triggeredWatch.triggerEvent(), defaultThrottlePeriod);
|
||||
executeAsync(ctx, triggeredWatch);
|
||||
counter++;
|
||||
try {
|
||||
Watch watch = parser.parseWithSecrets(response.getId(), true, response.getSourceAsBytesRef(), now, XContentType.JSON);
|
||||
TriggeredExecutionContext ctx =
|
||||
new StartupExecutionContext(watch, now, triggeredWatch.triggerEvent(), defaultThrottlePeriod);
|
||||
executeAsync(ctx, triggeredWatch);
|
||||
counter++;
|
||||
} catch (IOException e) {
|
||||
logger.error((Supplier<?>) () -> new ParameterizedMessage("Error parsing triggered watch"), e);
|
||||
}
|
||||
}
|
||||
}
|
||||
logger.debug("triggered execution of [{}] watches", counter);
|
||||
|
|
|
@ -22,6 +22,9 @@ public class QueuedWatch implements Streamable, ToXContent {
|
|||
private DateTime triggeredTime;
|
||||
private DateTime executionTime;
|
||||
|
||||
public QueuedWatch() {
|
||||
}
|
||||
|
||||
public QueuedWatch(WatchExecutionContext ctx) {
|
||||
this.watchId = ctx.watch().id();
|
||||
this.watchRecordId = ctx.id().value();
|
||||
|
@ -29,10 +32,6 @@ public class QueuedWatch implements Streamable, ToXContent {
|
|||
this.executionTime = ctx.executionTime();
|
||||
}
|
||||
|
||||
public QueuedWatch(StreamInput in) throws IOException {
|
||||
readFrom(in);
|
||||
}
|
||||
|
||||
public String watchId() {
|
||||
return watchId;
|
||||
}
|
||||
|
|
|
@ -41,9 +41,7 @@ public class TriggeredWatch implements ToXContent {
|
|||
@Override
|
||||
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
|
||||
builder.startObject();
|
||||
builder.startObject(Field.TRIGGER_EVENT.getPreferredName())
|
||||
.field(triggerEvent.type(), triggerEvent, params)
|
||||
.endObject();
|
||||
builder.field(Field.TRIGGER_EVENT.getPreferredName()).startObject().field(triggerEvent.type(), triggerEvent, params).endObject();
|
||||
builder.endObject();
|
||||
return builder;
|
||||
}
|
||||
|
|
|
@ -9,7 +9,6 @@ import org.apache.logging.log4j.message.ParameterizedMessage;
|
|||
import org.apache.logging.log4j.util.Supplier;
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.action.admin.indices.refresh.RefreshRequest;
|
||||
import org.elasticsearch.action.admin.indices.refresh.RefreshResponse;
|
||||
import org.elasticsearch.action.bulk.BulkItemResponse;
|
||||
import org.elasticsearch.action.bulk.BulkRequest;
|
||||
import org.elasticsearch.action.bulk.BulkResponse;
|
||||
|
@ -19,15 +18,18 @@ import org.elasticsearch.action.search.SearchRequest;
|
|||
import org.elasticsearch.action.search.SearchResponse;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
||||
import org.elasticsearch.cluster.routing.Preference;
|
||||
import org.elasticsearch.common.component.AbstractComponent;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.unit.TimeValue;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.common.xcontent.XContentFactory;
|
||||
import org.elasticsearch.index.IndexNotFoundException;
|
||||
import org.elasticsearch.search.SearchHit;
|
||||
import org.elasticsearch.search.builder.SearchSourceBuilder;
|
||||
import org.elasticsearch.search.sort.SortBuilders;
|
||||
import org.elasticsearch.xpack.watcher.support.init.proxy.WatcherClientProxy;
|
||||
import org.elasticsearch.xpack.watcher.watch.Watch;
|
||||
import org.elasticsearch.xpack.watcher.watch.WatchStoreUtils;
|
||||
|
||||
import java.io.IOException;
|
||||
|
@ -36,10 +38,12 @@ import java.util.BitSet;
|
|||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.util.concurrent.locks.Lock;
|
||||
import java.util.concurrent.locks.ReadWriteLock;
|
||||
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static org.elasticsearch.xpack.watcher.support.Exceptions.illegalState;
|
||||
import static org.elasticsearch.xpack.watcher.support.Exceptions.ioException;
|
||||
|
@ -61,10 +65,11 @@ public class TriggeredWatchStore extends AbstractComponent {
|
|||
|
||||
public TriggeredWatchStore(Settings settings, WatcherClientProxy client, TriggeredWatch.Parser triggeredWatchParser) {
|
||||
super(settings);
|
||||
this.scrollSize = settings.getAsInt("xpack.watcher.execution.scroll.size", 100);
|
||||
this.scrollSize = settings.getAsInt("xpack.watcher.execution.scroll.size", 1000);
|
||||
this.client = client;
|
||||
this.scrollTimeout = settings.getAsTime("xpack.watcher.execution.scroll.timeout", TimeValue.timeValueSeconds(30));
|
||||
this.triggeredWatchParser = triggeredWatchParser;
|
||||
this.started.set(true);
|
||||
}
|
||||
|
||||
public void start() {
|
||||
|
@ -74,7 +79,9 @@ public class TriggeredWatchStore extends AbstractComponent {
|
|||
public boolean validate(ClusterState state) {
|
||||
try {
|
||||
IndexMetaData indexMetaData = WatchStoreUtils.getConcreteIndex(INDEX_NAME, state.metaData());
|
||||
if (indexMetaData != null) {
|
||||
if (indexMetaData == null) {
|
||||
return true;
|
||||
} else {
|
||||
if (indexMetaData.getState() == IndexMetaData.State.CLOSE) {
|
||||
logger.debug("triggered watch index [{}] is marked as closed, watcher cannot be started",
|
||||
indexMetaData.getIndex().getName());
|
||||
|
@ -87,7 +94,6 @@ public class TriggeredWatchStore extends AbstractComponent {
|
|||
logger.trace((Supplier<?>) () -> new ParameterizedMessage("error getting index meta data [{}]: ", INDEX_NAME), e);
|
||||
return false;
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
public void stop() {
|
||||
|
@ -179,43 +185,53 @@ public class TriggeredWatchStore extends AbstractComponent {
|
|||
}
|
||||
}
|
||||
|
||||
public Collection<TriggeredWatch> loadTriggeredWatches(ClusterState state) {
|
||||
IndexMetaData indexMetaData = WatchStoreUtils.getConcreteIndex(INDEX_NAME, state.metaData());
|
||||
if (indexMetaData == null) {
|
||||
return Collections.emptySet();
|
||||
private void ensureStarted() {
|
||||
if (!started.get()) {
|
||||
throw illegalState("unable to persist triggered watches, the store is not ready");
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Checks if any of the loaded watches has been put into the triggered watches index for immediate execution
|
||||
*
|
||||
* Note: This is executing a blocking call over the network, thus a potential source of problems
|
||||
*
|
||||
* @param watches The list of watches that will be loaded here
|
||||
* @return A list of triggered watches that have been started to execute somewhere else but not finished
|
||||
*/
|
||||
public Collection<TriggeredWatch> findTriggeredWatches(Collection<Watch> watches) {
|
||||
if (watches.isEmpty()) {
|
||||
return Collections.emptyList();
|
||||
}
|
||||
|
||||
int numPrimaryShards;
|
||||
if (state.routingTable().index(indexMetaData.getIndex()).allPrimaryShardsActive() == false) {
|
||||
throw illegalState("not all primary shards of the triggered watches index {} are started", indexMetaData.getIndex());
|
||||
} else {
|
||||
numPrimaryShards = indexMetaData.getNumberOfShards();
|
||||
}
|
||||
RefreshResponse refreshResponse = client.refresh(new RefreshRequest(INDEX_NAME));
|
||||
if (refreshResponse.getSuccessfulShards() < numPrimaryShards) {
|
||||
throw illegalState("refresh was supposed to run on [{}] shards, but ran on [{}] shards", numPrimaryShards,
|
||||
refreshResponse.getSuccessfulShards());
|
||||
}
|
||||
|
||||
SearchRequest searchRequest = createScanSearchRequest();
|
||||
SearchResponse response = client.search(searchRequest, null);
|
||||
List<TriggeredWatch> triggeredWatches = new ArrayList<>();
|
||||
try {
|
||||
if (response.getTotalShards() != response.getSuccessfulShards()) {
|
||||
throw illegalState("scan search was supposed to run on [{}] shards, but ran on [{}] shards", numPrimaryShards,
|
||||
response.getSuccessfulShards());
|
||||
}
|
||||
client.refresh(new RefreshRequest(TriggeredWatchStore.INDEX_NAME));
|
||||
} catch (IndexNotFoundException e) {
|
||||
// no index, no problems, we dont need to search further
|
||||
return Collections.emptyList();
|
||||
}
|
||||
|
||||
Set<String> ids = watches.stream().map(Watch::id).collect(Collectors.toSet());
|
||||
Collection<TriggeredWatch> triggeredWatches = new ArrayList<>(ids.size());
|
||||
|
||||
SearchRequest searchRequest = new SearchRequest(TriggeredWatchStore.INDEX_NAME)
|
||||
.types(TriggeredWatchStore.DOC_TYPE)
|
||||
.scroll(scrollTimeout)
|
||||
.preference(Preference.LOCAL.toString())
|
||||
.source(new SearchSourceBuilder()
|
||||
.size(scrollSize)
|
||||
.sort(SortBuilders.fieldSort("_doc"))
|
||||
.version(true));
|
||||
|
||||
SearchResponse response = client.search(searchRequest);
|
||||
logger.debug("trying to find triggered watches for ids {}: found [{}] docs", ids, response.getHits().getTotalHits());
|
||||
try {
|
||||
while (response.getHits().getHits().length != 0) {
|
||||
for (SearchHit sh : response.getHits()) {
|
||||
String id = sh.getId();
|
||||
try {
|
||||
TriggeredWatch triggeredWatch = triggeredWatchParser.parse(id, sh.getVersion(), sh.getSourceRef());
|
||||
logger.trace("loaded triggered watch [{}/{}/{}]", sh.getIndex(), sh.getType(), sh.getId());
|
||||
for (SearchHit hit : response.getHits()) {
|
||||
Wid wid = new Wid(hit.getId());
|
||||
if (ids.contains(wid.watchId())) {
|
||||
TriggeredWatch triggeredWatch = triggeredWatchParser.parse(hit.getId(), hit.getVersion(), hit.getSourceRef());
|
||||
triggeredWatches.add(triggeredWatch);
|
||||
} catch (Exception e) {
|
||||
logger.error(
|
||||
(Supplier<?>) () -> new ParameterizedMessage("couldn't load triggered watch [{}], ignoring it...", id), e);
|
||||
}
|
||||
}
|
||||
response = client.searchScroll(response.getScrollId(), scrollTimeout);
|
||||
|
@ -223,26 +239,8 @@ public class TriggeredWatchStore extends AbstractComponent {
|
|||
} finally {
|
||||
client.clearScroll(response.getScrollId());
|
||||
}
|
||||
logger.debug("loaded [{}] triggered watches", triggeredWatches.size());
|
||||
|
||||
return triggeredWatches;
|
||||
}
|
||||
|
||||
private SearchRequest createScanSearchRequest() {
|
||||
SearchSourceBuilder sourceBuilder = new SearchSourceBuilder()
|
||||
.size(scrollSize)
|
||||
.sort(SortBuilders.fieldSort("_doc"));
|
||||
|
||||
SearchRequest searchRequest = new SearchRequest(INDEX_NAME);
|
||||
searchRequest.source(sourceBuilder);
|
||||
searchRequest.types(DOC_TYPE);
|
||||
searchRequest.scroll(scrollTimeout);
|
||||
searchRequest.preference("_primary");
|
||||
return searchRequest;
|
||||
}
|
||||
|
||||
private void ensureStarted() {
|
||||
if (!started.get()) {
|
||||
throw illegalState("unable to persist triggered watches, the store is not ready");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -40,6 +40,7 @@ public abstract class WatchExecutionContext {
|
|||
private Condition.Result conditionResult;
|
||||
private Transform.Result transformResult;
|
||||
private ConcurrentMap<String, ActionWrapper.Result> actionsResults = ConcurrentCollections.newConcurrentMap();
|
||||
private String nodeId;
|
||||
|
||||
public WatchExecutionContext(Watch watch, DateTime executionTime, TriggerEvent triggerEvent, TimeValue defaultThrottlePeriod) {
|
||||
this.id = new Wid(watch.id(), executionTime);
|
||||
|
@ -110,6 +111,20 @@ public abstract class WatchExecutionContext {
|
|||
return phase;
|
||||
}
|
||||
|
||||
/**
|
||||
* @param nodeId The node id this watch execution context runs on
|
||||
*/
|
||||
public void setNodeId(String nodeId) {
|
||||
this.nodeId = nodeId;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return The node this watch execution context runs on, which will be stored in the watch history
|
||||
*/
|
||||
public String getNodeId() {
|
||||
return nodeId;
|
||||
}
|
||||
|
||||
public void start() {
|
||||
assert phase == ExecutionPhase.AWAITS_EXECUTION;
|
||||
startTimestamp = System.currentTimeMillis();
|
||||
|
@ -183,7 +198,7 @@ public abstract class WatchExecutionContext {
|
|||
public WatchRecord abortBeforeExecution(ExecutionState state, String message) {
|
||||
assert !phase.sealed();
|
||||
phase = ExecutionPhase.ABORTED;
|
||||
return new WatchRecord.MessageWatchRecord(id, triggerEvent, state, message);
|
||||
return new WatchRecord.MessageWatchRecord(id, triggerEvent, state, message, getNodeId());
|
||||
}
|
||||
|
||||
public WatchRecord abortFailedExecution(String message) {
|
||||
|
@ -213,5 +228,4 @@ public abstract class WatchExecutionContext {
|
|||
public WatchExecutionSnapshot createSnapshot(Thread executionThread) {
|
||||
return new WatchExecutionSnapshot(this, executionThread.getStackTrace());
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -27,8 +27,7 @@ public class WatchExecutionSnapshot implements Streamable, ToXContentObject {
|
|||
private String[] executedActions;
|
||||
private StackTraceElement[] executionStackTrace;
|
||||
|
||||
public WatchExecutionSnapshot(StreamInput in) throws IOException {
|
||||
readFrom(in);
|
||||
public WatchExecutionSnapshot() {
|
||||
}
|
||||
|
||||
WatchExecutionSnapshot(WatchExecutionContext context, StackTraceElement[] executionStackTrace) {
|
||||
|
|
|
@ -5,7 +5,11 @@
|
|||
*/
|
||||
package org.elasticsearch.xpack.watcher.history;
|
||||
|
||||
import org.apache.logging.log4j.message.ParameterizedMessage;
|
||||
import org.apache.logging.log4j.util.Supplier;
|
||||
import org.elasticsearch.action.index.IndexRequest;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
||||
import org.elasticsearch.common.component.AbstractComponent;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.unit.TimeValue;
|
||||
|
@ -16,7 +20,9 @@ import org.elasticsearch.xpack.watcher.execution.ExecutionState;
|
|||
import org.elasticsearch.xpack.watcher.support.WatcherIndexTemplateRegistry;
|
||||
import org.elasticsearch.xpack.watcher.support.init.proxy.WatcherClientProxy;
|
||||
import org.elasticsearch.xpack.watcher.support.xcontent.WatcherParams;
|
||||
import org.elasticsearch.xpack.watcher.watch.WatchStoreUtils;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.DateTimeZone;
|
||||
import org.joda.time.format.DateTimeFormat;
|
||||
import org.joda.time.format.DateTimeFormatter;
|
||||
|
||||
|
@ -59,7 +65,6 @@ public class HistoryStore extends AbstractComponent {
|
|||
} finally {
|
||||
stopLock.unlock();
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -79,6 +84,7 @@ public class HistoryStore extends AbstractComponent {
|
|||
.source(builder)
|
||||
.opType(IndexRequest.OpType.CREATE);
|
||||
client.index(request, (TimeValue) null);
|
||||
logger.debug("indexed watch history record [{}]", watchRecord.id().value());
|
||||
} catch (IOException ioe) {
|
||||
throw ioException("failed to persist watch record [{}]", ioe, watchRecord);
|
||||
} finally {
|
||||
|
@ -90,29 +96,34 @@ public class HistoryStore extends AbstractComponent {
|
|||
* Stores the specified watchRecord.
|
||||
* Any existing watchRecord will be overwritten.
|
||||
*/
|
||||
public void forcePut(WatchRecord watchRecord) throws Exception {
|
||||
public void forcePut(WatchRecord watchRecord) {
|
||||
if (!started.get()) {
|
||||
throw new IllegalStateException("unable to persist watch record history store is not ready");
|
||||
}
|
||||
String index = getHistoryIndexNameForTime(watchRecord.triggerEvent().triggeredTime());
|
||||
putUpdateLock.lock();
|
||||
try (XContentBuilder builder = XContentFactory.jsonBuilder()) {
|
||||
watchRecord.toXContent(builder, WatcherParams.builder().hideSecrets(true).build());
|
||||
try {
|
||||
try (XContentBuilder builder = XContentFactory.jsonBuilder()) {
|
||||
watchRecord.toXContent(builder, WatcherParams.builder().hideSecrets(true).build());
|
||||
|
||||
IndexRequest request = new IndexRequest(index, DOC_TYPE, watchRecord.id().value())
|
||||
.source(builder)
|
||||
.opType(IndexRequest.OpType.CREATE);
|
||||
client.index(request, (TimeValue) null);
|
||||
} catch (VersionConflictEngineException vcee) {
|
||||
watchRecord = new WatchRecord.MessageWatchRecord(watchRecord, ExecutionState.EXECUTED_MULTIPLE_TIMES,
|
||||
"watch record [{ " + watchRecord.id() + " }] has been stored before, previous state [" + watchRecord.state() + "]");
|
||||
try (XContentBuilder xContentBuilder = XContentFactory.jsonBuilder()) {
|
||||
IndexRequest request = new IndexRequest(index, DOC_TYPE, watchRecord.id().value())
|
||||
.source(xContentBuilder.value(watchRecord));
|
||||
.source(builder)
|
||||
.opType(IndexRequest.OpType.CREATE);
|
||||
client.index(request, (TimeValue) null);
|
||||
logger.debug("indexed watch history record [{}]", watchRecord.id().value());
|
||||
} catch (VersionConflictEngineException vcee) {
|
||||
watchRecord = new WatchRecord.MessageWatchRecord(watchRecord, ExecutionState.EXECUTED_MULTIPLE_TIMES,
|
||||
"watch record [{ " + watchRecord.id() + " }] has been stored before, previous state [" + watchRecord.state() + "]");
|
||||
try (XContentBuilder xContentBuilder = XContentFactory.jsonBuilder()) {
|
||||
IndexRequest request = new IndexRequest(index, DOC_TYPE, watchRecord.id().value())
|
||||
.source(xContentBuilder.value(watchRecord));
|
||||
client.index(request, (TimeValue) null);
|
||||
}
|
||||
logger.debug("overwrote watch history record [{}]", watchRecord.id().value());
|
||||
}
|
||||
} catch (IOException ioe) {
|
||||
throw ioException("failed to persist watch record [{}]", ioe, watchRecord);
|
||||
final WatchRecord wr = watchRecord;
|
||||
logger.error((Supplier<?>) () -> new ParameterizedMessage("failed to persist watch record [{}]", wr), ioe);
|
||||
} finally {
|
||||
putUpdateLock.unlock();
|
||||
}
|
||||
|
@ -125,4 +136,21 @@ public class HistoryStore extends AbstractComponent {
|
|||
return INDEX_PREFIX_WITH_TEMPLATE + indexTimeFormat.print(time);
|
||||
}
|
||||
|
||||
/**
|
||||
* Check if everything is set up for the history store to operate fully. Checks for the
|
||||
* current watcher history index and if it is open.
|
||||
*
|
||||
* @param state The current cluster state
|
||||
* @return true, if history store is ready to be started
|
||||
*/
|
||||
public static boolean validate(ClusterState state) {
|
||||
String currentIndex = getHistoryIndexNameForTime(DateTime.now(DateTimeZone.UTC));
|
||||
IndexMetaData indexMetaData = WatchStoreUtils.getConcreteIndex(currentIndex, state.metaData());
|
||||
if (indexMetaData == null) {
|
||||
return true;
|
||||
} else {
|
||||
return indexMetaData.getState() == IndexMetaData.State.OPEN &&
|
||||
state.routingTable().index(indexMetaData.getIndex()).allPrimaryShardsActive();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -34,6 +34,7 @@ public abstract class WatchRecord implements ToXContentObject {
|
|||
|
||||
protected final Wid id;
|
||||
protected final Watch watch;
|
||||
private final String nodeId;
|
||||
protected final TriggerEvent triggerEvent;
|
||||
protected final ExecutionState state;
|
||||
|
||||
|
@ -46,7 +47,8 @@ public abstract class WatchRecord implements ToXContentObject {
|
|||
@Nullable protected final WatchExecutionResult executionResult;
|
||||
|
||||
private WatchRecord(Wid id, TriggerEvent triggerEvent, ExecutionState state, Map<String, Object> vars, ExecutableInput input,
|
||||
Condition condition, Map<String, Object> metadata, Watch watch, WatchExecutionResult executionResult) {
|
||||
Condition condition, Map<String, Object> metadata, Watch watch, WatchExecutionResult executionResult,
|
||||
String nodeId) {
|
||||
this.id = id;
|
||||
this.triggerEvent = triggerEvent;
|
||||
this.state = state;
|
||||
|
@ -56,25 +58,26 @@ public abstract class WatchRecord implements ToXContentObject {
|
|||
this.metadata = metadata;
|
||||
this.executionResult = executionResult;
|
||||
this.watch = watch;
|
||||
this.nodeId = nodeId;
|
||||
}
|
||||
|
||||
private WatchRecord(Wid id, TriggerEvent triggerEvent, ExecutionState state) {
|
||||
this(id, triggerEvent, state, Collections.emptyMap(), null, null, null, null, null);
|
||||
private WatchRecord(Wid id, TriggerEvent triggerEvent, ExecutionState state, String nodeId) {
|
||||
this(id, triggerEvent, state, Collections.emptyMap(), null, null, null, null, null, nodeId);
|
||||
}
|
||||
|
||||
private WatchRecord(WatchRecord record, ExecutionState state) {
|
||||
this(record.id, record.triggerEvent, state, record.vars, record.input, record.condition, record.metadata, record.watch,
|
||||
record.executionResult);
|
||||
record.executionResult, record.nodeId);
|
||||
}
|
||||
|
||||
private WatchRecord(WatchExecutionContext context, ExecutionState state) {
|
||||
this(context.id(), context.triggerEvent(), state, context.vars(), context.watch().input(), context.watch().condition(),
|
||||
context.watch().metadata(), context.watch(), null);
|
||||
context.watch().metadata(), context.watch(), null, context.getNodeId());
|
||||
}
|
||||
|
||||
private WatchRecord(WatchExecutionContext context, WatchExecutionResult executionResult) {
|
||||
this(context.id(), context.triggerEvent(), getState(executionResult), context.vars(), context.watch().input(),
|
||||
context.watch().condition(), context.watch().metadata(), context.watch(), executionResult);
|
||||
context.watch().condition(), context.watch().metadata(), context.watch(), executionResult, context.getNodeId());
|
||||
}
|
||||
|
||||
private static ExecutionState getState(WatchExecutionResult executionResult) {
|
||||
|
@ -122,10 +125,15 @@ public abstract class WatchRecord implements ToXContentObject {
|
|||
return executionResult;
|
||||
}
|
||||
|
||||
public String getNodeId() {
|
||||
return nodeId;
|
||||
}
|
||||
|
||||
@Override
|
||||
public final XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
|
||||
builder.startObject();
|
||||
builder.field(Field.WATCH_ID.getPreferredName(), id.watchId());
|
||||
builder.field(Field.NODE.getPreferredName(), nodeId);
|
||||
builder.field(Field.STATE.getPreferredName(), state.id());
|
||||
|
||||
if (watch != null && watch.status() != null) {
|
||||
|
@ -183,6 +191,7 @@ public abstract class WatchRecord implements ToXContentObject {
|
|||
|
||||
public interface Field {
|
||||
ParseField WATCH_ID = new ParseField("watch_id");
|
||||
ParseField NODE = new ParseField("node");
|
||||
ParseField TRIGGER_EVENT = new ParseField("trigger_event");
|
||||
ParseField MESSAGES = new ParseField("messages");
|
||||
ParseField STATE = new ParseField("state");
|
||||
|
@ -199,8 +208,8 @@ public abstract class WatchRecord implements ToXContentObject {
|
|||
/**
|
||||
* Called when the execution was aborted before it started
|
||||
*/
|
||||
public MessageWatchRecord(Wid id, TriggerEvent triggerEvent, ExecutionState state, String message) {
|
||||
super(id, triggerEvent, state);
|
||||
public MessageWatchRecord(Wid id, TriggerEvent triggerEvent, ExecutionState state, String message, String nodeId) {
|
||||
super(id, triggerEvent, state, nodeId);
|
||||
this.messages = new String[] { message };
|
||||
}
|
||||
|
||||
|
@ -221,7 +230,7 @@ public abstract class WatchRecord implements ToXContentObject {
|
|||
this.messages = Strings.EMPTY_ARRAY;
|
||||
}
|
||||
|
||||
public MessageWatchRecord(WatchRecord record, ExecutionState state, String message) {
|
||||
MessageWatchRecord(WatchRecord record, ExecutionState state, String message) {
|
||||
super(record, state);
|
||||
if (record instanceof MessageWatchRecord) {
|
||||
MessageWatchRecord messageWatchRecord = (MessageWatchRecord) record;
|
||||
|
|
|
@ -73,7 +73,7 @@ public class ExecutableSearchInput extends ExecutableInput<SearchInput, SearchIn
|
|||
if (logger.isDebugEnabled()) {
|
||||
logger.debug("[{}] found [{}] hits", ctx.id(), response.getHits().getTotalHits());
|
||||
for (SearchHit hit : response.getHits()) {
|
||||
logger.debug("[{}] hit [{}]", ctx.id(), XContentHelper.toString(hit));
|
||||
logger.debug("[{}] hit [{}]", ctx.id(), hit.getSourceAsMap());
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -13,7 +13,6 @@ import org.elasticsearch.common.xcontent.XContentParser;
|
|||
import org.elasticsearch.script.ScriptService;
|
||||
import org.elasticsearch.xpack.security.InternalClient;
|
||||
import org.elasticsearch.xpack.watcher.input.InputFactory;
|
||||
import org.elasticsearch.xpack.watcher.input.simple.ExecutableSimpleInput;
|
||||
import org.elasticsearch.xpack.watcher.support.init.proxy.WatcherClientProxy;
|
||||
import org.elasticsearch.xpack.watcher.support.search.WatcherSearchTemplateService;
|
||||
|
||||
|
@ -31,7 +30,7 @@ public class SearchInputFactory extends InputFactory<SearchInput, SearchInput.Re
|
|||
|
||||
public SearchInputFactory(Settings settings, WatcherClientProxy client, NamedXContentRegistry xContentRegistry,
|
||||
ScriptService scriptService) {
|
||||
super(Loggers.getLogger(ExecutableSimpleInput.class, settings));
|
||||
super(Loggers.getLogger(ExecutableSearchInput.class, settings));
|
||||
this.client = client;
|
||||
this.defaultTimeout = settings.getAsTime("xpack.watcher.input.search.default_timeout", null);
|
||||
this.searchTemplateService = new WatcherSearchTemplateService(settings, scriptService, xContentRegistry);
|
||||
|
|
|
@ -5,7 +5,9 @@
|
|||
*/
|
||||
package org.elasticsearch.xpack.watcher.rest.action;
|
||||
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.rest.BytesRestResponse;
|
||||
import org.elasticsearch.rest.RestController;
|
||||
import org.elasticsearch.rest.RestRequest;
|
||||
import org.elasticsearch.rest.action.AcknowledgedRestListener;
|
||||
|
@ -19,22 +21,35 @@ import static org.elasticsearch.rest.RestRequest.Method.POST;
|
|||
import static org.elasticsearch.rest.RestRequest.Method.PUT;
|
||||
|
||||
public class RestWatchServiceAction extends WatcherRestHandler {
|
||||
|
||||
public RestWatchServiceAction(Settings settings, RestController controller) {
|
||||
super(settings);
|
||||
|
||||
// @deprecated Remove in 6.0
|
||||
// NOTE: we switched from PUT in 2.x to POST in 5.x
|
||||
controller.registerWithDeprecatedHandler(POST, URI_BASE + "/_restart", this,
|
||||
PUT, "/_watcher/_restart", deprecationLogger);
|
||||
controller.registerWithDeprecatedHandler(POST, URI_BASE + "/_start", new StartRestHandler(settings),
|
||||
PUT, "/_watcher/_start", deprecationLogger);
|
||||
controller.registerWithDeprecatedHandler(POST, URI_BASE + "/_stop", new StopRestHandler(settings),
|
||||
PUT, "/_watcher/_stop", deprecationLogger);
|
||||
PUT, "/_watcher/_restart", deprecationLogger);
|
||||
controller.registerWithDeprecatedHandler(POST, URI_BASE + "/_start",
|
||||
new StartRestHandler(settings), PUT, "/_watcher/_start", deprecationLogger);
|
||||
controller.registerWithDeprecatedHandler(POST, URI_BASE + "/_stop",
|
||||
new StopRestHandler(settings), PUT, "/_watcher/_stop", deprecationLogger);
|
||||
}
|
||||
|
||||
@Override
|
||||
public RestChannelConsumer doPrepareRequest(RestRequest request, WatcherClient client) throws IOException {
|
||||
return channel -> client.watcherService(new WatcherServiceRequest().restart(), new AcknowledgedRestListener<>(channel));
|
||||
public RestChannelConsumer doPrepareRequest(RestRequest request, WatcherClient client)
|
||||
throws IOException {
|
||||
return channel -> client.watcherService(new WatcherServiceRequest().stop(),
|
||||
ActionListener.wrap(
|
||||
stopResponse -> client.watcherService(new WatcherServiceRequest().start(),
|
||||
new AcknowledgedRestListener<>(channel))
|
||||
, e -> {
|
||||
try {
|
||||
channel.sendResponse(new BytesRestResponse(channel, e));
|
||||
} catch (Exception inner) {
|
||||
inner.addSuppressed(e);
|
||||
logger.error("failed to send failure response", inner);
|
||||
}
|
||||
}));
|
||||
}
|
||||
|
||||
private static class StartRestHandler extends WatcherRestHandler {
|
||||
|
@ -44,10 +59,11 @@ public class RestWatchServiceAction extends WatcherRestHandler {
|
|||
}
|
||||
|
||||
@Override
|
||||
public RestChannelConsumer doPrepareRequest(RestRequest request, WatcherClient client) throws IOException {
|
||||
return channel -> client.watcherService(new WatcherServiceRequest().start(), new AcknowledgedRestListener<>(channel));
|
||||
public RestChannelConsumer doPrepareRequest(RestRequest request, WatcherClient client)
|
||||
throws IOException {
|
||||
return channel -> client.watcherService(new WatcherServiceRequest().start(),
|
||||
new AcknowledgedRestListener<>(channel));
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
private static class StopRestHandler extends WatcherRestHandler {
|
||||
|
@ -57,10 +73,10 @@ public class RestWatchServiceAction extends WatcherRestHandler {
|
|||
}
|
||||
|
||||
@Override
|
||||
public RestChannelConsumer doPrepareRequest(RestRequest request, WatcherClient client) throws IOException {
|
||||
return channel -> client.watcherService(new WatcherServiceRequest().stop(), new AcknowledgedRestListener<>(channel));
|
||||
public RestChannelConsumer doPrepareRequest(RestRequest request, WatcherClient client)
|
||||
throws IOException {
|
||||
return channel -> client.watcherService(new WatcherServiceRequest().stop(), new
|
||||
AcknowledgedRestListener<>(channel));
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -7,12 +7,9 @@ package org.elasticsearch.xpack.watcher.rest.action;
|
|||
|
||||
import org.elasticsearch.common.Strings;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.rest.BytesRestResponse;
|
||||
import org.elasticsearch.rest.RestController;
|
||||
import org.elasticsearch.rest.RestRequest;
|
||||
import org.elasticsearch.rest.RestResponse;
|
||||
import org.elasticsearch.rest.action.RestBuilderListener;
|
||||
import org.elasticsearch.rest.action.RestActions;
|
||||
import org.elasticsearch.xpack.watcher.client.WatcherClient;
|
||||
import org.elasticsearch.xpack.watcher.rest.WatcherRestHandler;
|
||||
import org.elasticsearch.xpack.watcher.transport.actions.stats.WatcherStatsRequest;
|
||||
|
@ -22,7 +19,6 @@ import java.io.IOException;
|
|||
import java.util.Set;
|
||||
|
||||
import static org.elasticsearch.rest.RestRequest.Method.GET;
|
||||
import static org.elasticsearch.rest.RestStatus.OK;
|
||||
|
||||
public class RestWatcherStatsAction extends WatcherRestHandler {
|
||||
public RestWatcherStatsAction(Settings settings, RestController controller) {
|
||||
|
@ -48,12 +44,7 @@ public class RestWatcherStatsAction extends WatcherRestHandler {
|
|||
request.includeQueuedWatches(metrics.contains("pending_watches"));
|
||||
}
|
||||
|
||||
return channel -> client.watcherStats(request, new RestBuilderListener<WatcherStatsResponse>(channel) {
|
||||
@Override
|
||||
public RestResponse buildResponse(WatcherStatsResponse watcherStatsResponse, XContentBuilder builder) throws Exception {
|
||||
watcherStatsResponse.toXContent(builder, restRequest);
|
||||
return new BytesRestResponse(OK, builder);
|
||||
}
|
||||
});
|
||||
|
||||
return channel -> client.watcherStats(request, new RestActions.NodesResponseRestListener<>(channel));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -37,7 +37,7 @@ import static java.util.Collections.unmodifiableMap;
|
|||
public class WatcherIndexTemplateRegistry extends AbstractComponent implements ClusterStateListener {
|
||||
|
||||
private static final String FORBIDDEN_INDEX_SETTING = "index.mapper.dynamic";
|
||||
public static final String INDEX_TEMPLATE_VERSION = "3";
|
||||
public static final String INDEX_TEMPLATE_VERSION = "4";
|
||||
|
||||
public static final String HISTORY_TEMPLATE_NAME = "watch_history_" + INDEX_TEMPLATE_VERSION;
|
||||
public static final String TRIGGERED_TEMPLATE_NAME = "triggered_watches";
|
||||
|
@ -95,23 +95,14 @@ public class WatcherIndexTemplateRegistry extends AbstractComponent implements C
|
|||
return;
|
||||
}
|
||||
|
||||
addTemplatesIfMissing(state, false);
|
||||
addTemplatesIfMissing(state);
|
||||
}
|
||||
|
||||
/**
|
||||
* Adds the registered index templates if missing to the cluster.
|
||||
*/
|
||||
public void addTemplatesIfMissing() {
|
||||
// to be sure that the templates exist after this method call, we should wait until the put index templates calls
|
||||
// have returned if the templates were missing
|
||||
addTemplatesIfMissing(clusterService.state(), true);
|
||||
}
|
||||
|
||||
void addTemplatesIfMissing(ClusterState state, boolean wait) {
|
||||
void addTemplatesIfMissing(ClusterState state) {
|
||||
for (TemplateConfig template : indexTemplates) {
|
||||
if (!state.metaData().getTemplates().containsKey(template.getTemplateName())) {
|
||||
logger.debug("adding index template [{}], because it doesn't exist", template.getTemplateName());
|
||||
putTemplate(template, wait);
|
||||
putTemplate(template);
|
||||
} else {
|
||||
logger.trace("not adding index template [{}], because it already exists", template.getTemplateName());
|
||||
}
|
||||
|
@ -155,17 +146,12 @@ public class WatcherIndexTemplateRegistry extends AbstractComponent implements C
|
|||
Map<String, Settings> customIndexSettings = new HashMap<String, Settings>(this.customIndexSettings);
|
||||
customIndexSettings.put(config.getSetting().getKey(), builder.build());
|
||||
this.customIndexSettings = customIndexSettings;
|
||||
putTemplate(config, false);
|
||||
putTemplate(config);
|
||||
}
|
||||
}
|
||||
|
||||
private void putTemplate(final TemplateConfig config, boolean wait) {
|
||||
final Executor executor;
|
||||
if (wait) {
|
||||
executor = Runnable::run;
|
||||
} else {
|
||||
executor = threadPool.generic();
|
||||
}
|
||||
private void putTemplate(final TemplateConfig config) {
|
||||
final Executor executor = threadPool.generic();
|
||||
executor.execute(() -> {
|
||||
final byte[] template = TemplateUtils.loadTemplate("/" + config.getFileName()+ ".json", INDEX_TEMPLATE_VERSION,
|
||||
Pattern.quote("${xpack.watcher.template.version}")).getBytes(StandardCharsets.UTF_8);
|
||||
|
|
|
@ -101,6 +101,10 @@ public class WatcherClientProxy extends ClientProxy {
|
|||
return client.delete(preProcess(request)).actionGet(defaultIndexTimeout);
|
||||
}
|
||||
|
||||
public SearchResponse search(SearchRequest request) {
|
||||
return search(request, defaultSearchTimeout);
|
||||
}
|
||||
|
||||
public SearchResponse search(SearchRequest request, TimeValue timeout) {
|
||||
if (timeout == null) {
|
||||
timeout = defaultSearchTimeout;
|
||||
|
|
|
@ -5,15 +5,12 @@
|
|||
*/
|
||||
package org.elasticsearch.xpack.watcher.transport.actions;
|
||||
|
||||
import java.util.function.Supplier;
|
||||
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.action.ActionRequest;
|
||||
import org.elasticsearch.action.ActionResponse;
|
||||
import org.elasticsearch.action.support.ActionFilters;
|
||||
import org.elasticsearch.action.support.master.MasterNodeRequest;
|
||||
import org.elasticsearch.action.support.master.TransportMasterNodeAction;
|
||||
import org.elasticsearch.action.support.HandledTransportAction;
|
||||
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
|
||||
import org.elasticsearch.cluster.service.ClusterService;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.license.LicenseUtils;
|
||||
import org.elasticsearch.license.XPackLicenseState;
|
||||
|
@ -22,16 +19,17 @@ import org.elasticsearch.threadpool.ThreadPool;
|
|||
import org.elasticsearch.transport.TransportService;
|
||||
import org.elasticsearch.xpack.XPackPlugin;
|
||||
|
||||
public abstract class WatcherTransportAction<Request extends MasterNodeRequest<Request>, Response extends ActionResponse>
|
||||
extends TransportMasterNodeAction<Request, Response> {
|
||||
import java.util.function.Supplier;
|
||||
|
||||
public abstract class WatcherTransportAction<Request extends ActionRequest, Response extends ActionResponse>
|
||||
extends HandledTransportAction<Request, Response> {
|
||||
|
||||
protected final XPackLicenseState licenseState;
|
||||
|
||||
public WatcherTransportAction(Settings settings, String actionName, TransportService transportService,
|
||||
ClusterService clusterService, ThreadPool threadPool, ActionFilters actionFilters,
|
||||
IndexNameExpressionResolver indexNameExpressionResolver, XPackLicenseState licenseState,
|
||||
Supplier<Request> request) {
|
||||
super(settings, actionName, transportService, clusterService, threadPool, actionFilters, indexNameExpressionResolver, request);
|
||||
public WatcherTransportAction(Settings settings, String actionName, TransportService transportService, ThreadPool threadPool,
|
||||
ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver,
|
||||
XPackLicenseState licenseState, Supplier<Request> request) {
|
||||
super(settings, actionName, threadPool, transportService, actionFilters, indexNameExpressionResolver, request);
|
||||
this.licenseState = licenseState;
|
||||
}
|
||||
|
||||
|
|
|
@ -5,17 +5,12 @@
|
|||
*/
|
||||
package org.elasticsearch.xpack.watcher.transport.actions.ack;
|
||||
|
||||
import org.elasticsearch.ElasticsearchException;
|
||||
import org.elasticsearch.ResourceNotFoundException;
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.action.support.ActionFilters;
|
||||
import org.elasticsearch.action.support.WriteRequest;
|
||||
import org.elasticsearch.action.update.UpdateRequest;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
import org.elasticsearch.cluster.block.ClusterBlockException;
|
||||
import org.elasticsearch.cluster.block.ClusterBlockLevel;
|
||||
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
|
||||
import org.elasticsearch.cluster.service.ClusterService;
|
||||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.xcontent.ToXContent;
|
||||
|
@ -37,9 +32,6 @@ import java.util.List;
|
|||
import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
|
||||
import static org.joda.time.DateTimeZone.UTC;
|
||||
|
||||
/**
|
||||
* Performs the ack operation.
|
||||
*/
|
||||
public class TransportAckWatchAction extends WatcherTransportAction<AckWatchRequest, AckWatchResponse> {
|
||||
|
||||
private final Clock clock;
|
||||
|
@ -47,11 +39,10 @@ public class TransportAckWatchAction extends WatcherTransportAction<AckWatchRequ
|
|||
private final WatcherClientProxy client;
|
||||
|
||||
@Inject
|
||||
public TransportAckWatchAction(Settings settings, TransportService transportService, ClusterService clusterService,
|
||||
ThreadPool threadPool, ActionFilters actionFilters,
|
||||
public TransportAckWatchAction(Settings settings, TransportService transportService, ThreadPool threadPool, ActionFilters actionFilters,
|
||||
IndexNameExpressionResolver indexNameExpressionResolver, Clock clock, XPackLicenseState licenseState,
|
||||
Watch.Parser parser, WatcherClientProxy client) {
|
||||
super(settings, AckWatchAction.NAME, transportService, clusterService, threadPool, actionFilters, indexNameExpressionResolver,
|
||||
super(settings, AckWatchAction.NAME, transportService, threadPool, actionFilters, indexNameExpressionResolver,
|
||||
licenseState, AckWatchRequest::new);
|
||||
this.clock = clock;
|
||||
this.parser = parser;
|
||||
|
@ -59,18 +50,7 @@ public class TransportAckWatchAction extends WatcherTransportAction<AckWatchRequ
|
|||
}
|
||||
|
||||
@Override
|
||||
protected String executor() {
|
||||
return ThreadPool.Names.MANAGEMENT;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected AckWatchResponse newResponse() {
|
||||
return new AckWatchResponse();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void masterOperation(AckWatchRequest request, ClusterState state, ActionListener<AckWatchResponse> listener) throws
|
||||
ElasticsearchException {
|
||||
protected void doExecute(AckWatchRequest request, ActionListener<AckWatchResponse> listener) {
|
||||
client.getWatch(request.getWatchId(), ActionListener.wrap((response) -> {
|
||||
if (response.isExists() == false) {
|
||||
listener.onFailure(new ResourceNotFoundException("Watch with id [{}] does not exit", request.getWatchId()));
|
||||
|
@ -119,9 +99,4 @@ public class TransportAckWatchAction extends WatcherTransportAction<AckWatchRequ
|
|||
}
|
||||
}, listener::onFailure));
|
||||
}
|
||||
|
||||
@Override
|
||||
protected ClusterBlockException checkBlock(AckWatchRequest request, ClusterState state) {
|
||||
return state.blocks().indexBlockedException(ClusterBlockLevel.WRITE, Watch.INDEX);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -5,17 +5,12 @@
|
|||
*/
|
||||
package org.elasticsearch.xpack.watcher.transport.actions.activate;
|
||||
|
||||
import org.elasticsearch.ElasticsearchException;
|
||||
import org.elasticsearch.ResourceNotFoundException;
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.action.support.ActionFilters;
|
||||
import org.elasticsearch.action.support.WriteRequest;
|
||||
import org.elasticsearch.action.update.UpdateRequest;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
import org.elasticsearch.cluster.block.ClusterBlockException;
|
||||
import org.elasticsearch.cluster.block.ClusterBlockLevel;
|
||||
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
|
||||
import org.elasticsearch.cluster.service.ClusterService;
|
||||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
|
@ -25,7 +20,6 @@ import org.elasticsearch.threadpool.ThreadPool;
|
|||
import org.elasticsearch.transport.TransportService;
|
||||
import org.elasticsearch.xpack.watcher.support.init.proxy.WatcherClientProxy;
|
||||
import org.elasticsearch.xpack.watcher.transport.actions.WatcherTransportAction;
|
||||
import org.elasticsearch.xpack.watcher.trigger.TriggerService;
|
||||
import org.elasticsearch.xpack.watcher.watch.Watch;
|
||||
import org.elasticsearch.xpack.watcher.watch.WatchStatus;
|
||||
import org.joda.time.DateTime;
|
||||
|
@ -43,48 +37,33 @@ import static org.joda.time.DateTimeZone.UTC;
|
|||
public class TransportActivateWatchAction extends WatcherTransportAction<ActivateWatchRequest, ActivateWatchResponse> {
|
||||
|
||||
private final Clock clock;
|
||||
private final TriggerService triggerService;
|
||||
private final Watch.Parser parser;
|
||||
private final WatcherClientProxy client;
|
||||
|
||||
@Inject
|
||||
public TransportActivateWatchAction(Settings settings, TransportService transportService, ClusterService clusterService,
|
||||
ThreadPool threadPool, ActionFilters actionFilters,
|
||||
IndexNameExpressionResolver indexNameExpressionResolver, Clock clock,
|
||||
XPackLicenseState licenseState, TriggerService triggerService, Watch.Parser parser,
|
||||
public TransportActivateWatchAction(Settings settings, TransportService transportService, ThreadPool threadPool,
|
||||
ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver, Clock clock,
|
||||
XPackLicenseState licenseState, Watch.Parser parser,
|
||||
WatcherClientProxy client) {
|
||||
super(settings, ActivateWatchAction.NAME, transportService, clusterService, threadPool, actionFilters, indexNameExpressionResolver,
|
||||
super(settings, ActivateWatchAction.NAME, transportService, threadPool, actionFilters, indexNameExpressionResolver,
|
||||
licenseState, ActivateWatchRequest::new);
|
||||
this.clock = clock;
|
||||
this.triggerService = triggerService;
|
||||
this.parser = parser;
|
||||
this.client = client;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected String executor() {
|
||||
return ThreadPool.Names.MANAGEMENT;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected ActivateWatchResponse newResponse() {
|
||||
return new ActivateWatchResponse();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void masterOperation(ActivateWatchRequest request, ClusterState state, ActionListener<ActivateWatchResponse> listener)
|
||||
throws ElasticsearchException {
|
||||
protected void doExecute(ActivateWatchRequest request, ActionListener<ActivateWatchResponse> listener) {
|
||||
try {
|
||||
// if this is about deactivation, remove this immediately from the trigger service, no need to wait for all those async calls
|
||||
if (request.isActivate() == false) {
|
||||
triggerService.remove(request.getWatchId());
|
||||
}
|
||||
|
||||
DateTime now = new DateTime(clock.millis(), UTC);
|
||||
UpdateRequest updateRequest = new UpdateRequest(Watch.INDEX, Watch.DOC_TYPE, request.getWatchId());
|
||||
updateRequest.setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE);
|
||||
XContentBuilder builder = activateWatchBuilder(request.isActivate(), now);
|
||||
updateRequest.doc(builder);
|
||||
// a watch execution updates the status in between, we still want this want to override the active state
|
||||
// two has been chosen arbitrary, maybe one would make more sense, as a watch would not execute more often than
|
||||
// once per second?
|
||||
updateRequest.retryOnConflict(2);
|
||||
|
||||
client.update(updateRequest, ActionListener.wrap(updateResponse -> {
|
||||
client.getWatch(request.getWatchId(), ActionListener.wrap(getResponse -> {
|
||||
|
@ -93,10 +72,6 @@ public class TransportActivateWatchAction extends WatcherTransportAction<Activat
|
|||
XContentType.JSON);
|
||||
watch.version(getResponse.getVersion());
|
||||
watch.status().version(getResponse.getVersion());
|
||||
|
||||
if (request.isActivate()) {
|
||||
triggerService.add(watch);
|
||||
}
|
||||
listener.onResponse(new ActivateWatchResponse(watch.status()));
|
||||
} else {
|
||||
listener.onFailure(new ResourceNotFoundException("Watch with id [{}] does not exist", request.getWatchId()));
|
||||
|
@ -120,9 +95,4 @@ public class TransportActivateWatchAction extends WatcherTransportAction<Activat
|
|||
return builder;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected ClusterBlockException checkBlock(ActivateWatchRequest request, ClusterState state) {
|
||||
return state.blocks().indexBlockedException(ClusterBlockLevel.WRITE, Watch.INDEX);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -5,70 +5,41 @@
|
|||
*/
|
||||
package org.elasticsearch.xpack.watcher.transport.actions.delete;
|
||||
|
||||
import org.elasticsearch.ElasticsearchException;
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.action.DocWriteResponse;
|
||||
import org.elasticsearch.action.support.ActionFilters;
|
||||
import org.elasticsearch.action.support.master.TransportMasterNodeAction;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
import org.elasticsearch.cluster.block.ClusterBlockException;
|
||||
import org.elasticsearch.cluster.block.ClusterBlockLevel;
|
||||
import org.elasticsearch.action.support.HandledTransportAction;
|
||||
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
|
||||
import org.elasticsearch.cluster.service.ClusterService;
|
||||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
import org.elasticsearch.transport.TransportService;
|
||||
import org.elasticsearch.xpack.watcher.support.init.proxy.WatcherClientProxy;
|
||||
import org.elasticsearch.xpack.watcher.trigger.TriggerService;
|
||||
import org.elasticsearch.xpack.watcher.watch.Watch;
|
||||
|
||||
/**
|
||||
* Performs the delete operation. This inherits directly from TransportMasterNodeAction, because deletion should always work
|
||||
* Performs the delete operation. This inherits directly from HandledTransportAction, because deletion should always work
|
||||
* independently from the license check in WatcherTransportAction!
|
||||
*/
|
||||
public class TransportDeleteWatchAction extends TransportMasterNodeAction<DeleteWatchRequest, DeleteWatchResponse> {
|
||||
public class TransportDeleteWatchAction extends HandledTransportAction<DeleteWatchRequest, DeleteWatchResponse> {
|
||||
|
||||
private final WatcherClientProxy client;
|
||||
private final TriggerService triggerService;
|
||||
|
||||
@Inject
|
||||
public TransportDeleteWatchAction(Settings settings, TransportService transportService, ClusterService clusterService,
|
||||
ThreadPool threadPool, ActionFilters actionFilters,
|
||||
IndexNameExpressionResolver indexNameExpressionResolver, WatcherClientProxy client,
|
||||
TriggerService triggerService) {
|
||||
super(settings, DeleteWatchAction.NAME, transportService, clusterService, threadPool, actionFilters, indexNameExpressionResolver,
|
||||
public TransportDeleteWatchAction(Settings settings, TransportService transportService,ThreadPool threadPool,
|
||||
ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver,
|
||||
WatcherClientProxy client) {
|
||||
super(settings, DeleteWatchAction.NAME, threadPool, transportService, actionFilters, indexNameExpressionResolver,
|
||||
DeleteWatchRequest::new);
|
||||
this.client = client;
|
||||
this.triggerService = triggerService;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected String executor() {
|
||||
return ThreadPool.Names.MANAGEMENT;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected DeleteWatchResponse newResponse() {
|
||||
return new DeleteWatchResponse();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void masterOperation(DeleteWatchRequest request, ClusterState state, ActionListener<DeleteWatchResponse> listener) throws
|
||||
ElasticsearchException {
|
||||
protected void doExecute(DeleteWatchRequest request, ActionListener<DeleteWatchResponse> listener) {
|
||||
client.deleteWatch(request.getId(), ActionListener.wrap(deleteResponse -> {
|
||||
boolean deleted = deleteResponse.getResult() == DocWriteResponse.Result.DELETED;
|
||||
DeleteWatchResponse response = new DeleteWatchResponse(deleteResponse.getId(), deleteResponse.getVersion(), deleted);
|
||||
if (deleted) {
|
||||
triggerService.remove(request.getId());
|
||||
}
|
||||
listener.onResponse(response);
|
||||
},
|
||||
listener::onFailure));
|
||||
}
|
||||
|
||||
@Override
|
||||
protected ClusterBlockException checkBlock(DeleteWatchRequest request, ClusterState state) {
|
||||
return state.blocks().indexBlockedException(ClusterBlockLevel.WRITE, Watch.INDEX);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -7,15 +7,9 @@ package org.elasticsearch.xpack.watcher.transport.actions.execute;
|
|||
|
||||
import org.apache.logging.log4j.message.ParameterizedMessage;
|
||||
import org.apache.logging.log4j.util.Supplier;
|
||||
import org.elasticsearch.ElasticsearchException;
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.action.get.GetResponse;
|
||||
import org.elasticsearch.action.support.ActionFilters;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
import org.elasticsearch.cluster.block.ClusterBlockException;
|
||||
import org.elasticsearch.cluster.block.ClusterBlockLevel;
|
||||
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
|
||||
import org.elasticsearch.cluster.service.ClusterService;
|
||||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
|
@ -24,6 +18,7 @@ import org.elasticsearch.common.xcontent.XContentType;
|
|||
import org.elasticsearch.license.XPackLicenseState;
|
||||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
import org.elasticsearch.transport.TransportService;
|
||||
import org.elasticsearch.xpack.XPackPlugin;
|
||||
import org.elasticsearch.xpack.watcher.condition.AlwaysCondition;
|
||||
import org.elasticsearch.xpack.watcher.execution.ActionExecutionMode;
|
||||
import org.elasticsearch.xpack.watcher.execution.ExecutionService;
|
||||
|
@ -58,12 +53,11 @@ public class TransportExecuteWatchAction extends WatcherTransportAction<ExecuteW
|
|||
private final WatcherClientProxy client;
|
||||
|
||||
@Inject
|
||||
public TransportExecuteWatchAction(Settings settings, TransportService transportService, ClusterService clusterService,
|
||||
ThreadPool threadPool, ActionFilters actionFilters,
|
||||
IndexNameExpressionResolver indexNameExpressionResolver, ExecutionService executionService,
|
||||
Clock clock, XPackLicenseState licenseState, TriggerService triggerService,
|
||||
Watch.Parser watchParser, WatcherClientProxy client) {
|
||||
super(settings, ExecuteWatchAction.NAME, transportService, clusterService, threadPool, actionFilters, indexNameExpressionResolver,
|
||||
public TransportExecuteWatchAction(Settings settings, TransportService transportService, ThreadPool threadPool,
|
||||
ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver,
|
||||
ExecutionService executionService, Clock clock, XPackLicenseState licenseState,
|
||||
Watch.Parser watchParser, WatcherClientProxy client, TriggerService triggerService) {
|
||||
super(settings, ExecuteWatchAction.NAME, transportService, threadPool, actionFilters, indexNameExpressionResolver,
|
||||
licenseState, ExecuteWatchRequest::new);
|
||||
this.executionService = executionService;
|
||||
this.clock = clock;
|
||||
|
@ -73,39 +67,22 @@ public class TransportExecuteWatchAction extends WatcherTransportAction<ExecuteW
|
|||
}
|
||||
|
||||
@Override
|
||||
protected String executor() {
|
||||
return ThreadPool.Names.MANAGEMENT;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected ExecuteWatchResponse newResponse() {
|
||||
return new ExecuteWatchResponse();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void masterOperation(ExecuteWatchRequest request, ClusterState state, ActionListener<ExecuteWatchResponse> listener)
|
||||
throws ElasticsearchException {
|
||||
protected void doExecute(ExecuteWatchRequest request, ActionListener<ExecuteWatchResponse> listener) {
|
||||
if (request.getId() != null) {
|
||||
try {
|
||||
// should be executed async in the future
|
||||
GetResponse getResponse = client.getWatch(request.getId());
|
||||
Watch watch = watchParser.parse(request.getId(), true, getResponse.getSourceAsBytesRef(), XContentType.JSON);
|
||||
watch.version(getResponse.getVersion());
|
||||
watch.status().version(getResponse.getVersion());
|
||||
ExecuteWatchResponse executeWatchResponse = executeWatch(request, watch, true);
|
||||
listener.onResponse(executeWatchResponse);
|
||||
} catch (IOException e) {
|
||||
listener.onFailure(e);
|
||||
}
|
||||
client.getWatch(request.getId(), ActionListener.wrap(response -> {
|
||||
Watch watch = watchParser.parse(request.getId(), true, response.getSourceAsBytesRef(), request.getXContentType());
|
||||
watch.version(response.getVersion());
|
||||
watch.status().version(response.getVersion());
|
||||
executeWatch(request, listener, watch, true);
|
||||
}, listener::onFailure));
|
||||
} else if (request.getWatchSource() != null) {
|
||||
try {
|
||||
assert !request.isRecordExecution();
|
||||
Watch watch =
|
||||
watchParser.parse(ExecuteWatchRequest.INLINE_WATCH_ID, true, request.getWatchSource(), request.getXContentType());
|
||||
ExecuteWatchResponse response = executeWatch(request, watch, false);
|
||||
listener.onResponse(response);
|
||||
} catch (Exception e) {
|
||||
logger.error((Supplier<?>) () -> new ParameterizedMessage("failed to execute [{}]", request.getId()), e);
|
||||
Watch watch = watchParser.parse(ExecuteWatchRequest.INLINE_WATCH_ID, true, request.getWatchSource(),
|
||||
request.getXContentType());
|
||||
executeWatch(request, listener, watch, false);
|
||||
} catch (IOException e) {
|
||||
logger.error((Supplier<?>) () -> new ParameterizedMessage("failed to parse [{}]", request.getId()), e);
|
||||
listener.onFailure(e);
|
||||
}
|
||||
} else {
|
||||
|
@ -113,37 +90,38 @@ public class TransportExecuteWatchAction extends WatcherTransportAction<ExecuteW
|
|||
}
|
||||
}
|
||||
|
||||
private ExecuteWatchResponse executeWatch(ExecuteWatchRequest request, Watch watch, boolean knownWatch) throws IOException {
|
||||
String triggerType = watch.trigger().type();
|
||||
TriggerEvent triggerEvent = triggerService.simulateEvent(triggerType, watch.id(), request.getTriggerData());
|
||||
private void executeWatch(ExecuteWatchRequest request, ActionListener<ExecuteWatchResponse> listener,
|
||||
Watch watch, boolean knownWatch) {
|
||||
|
||||
ManualExecutionContext.Builder ctxBuilder = ManualExecutionContext.builder(watch, knownWatch,
|
||||
new ManualTriggerEvent(triggerEvent.jobName(), triggerEvent), executionService.defaultThrottlePeriod());
|
||||
threadPool.executor(XPackPlugin.WATCHER).submit(() -> {
|
||||
try {
|
||||
String triggerType = watch.trigger().type();
|
||||
TriggerEvent triggerEvent = triggerService.simulateEvent(triggerType, watch.id(), request.getTriggerData());
|
||||
|
||||
DateTime executionTime = new DateTime(clock.millis(), UTC);
|
||||
ctxBuilder.executionTime(executionTime);
|
||||
for (Map.Entry<String, ActionExecutionMode> entry : request.getActionModes().entrySet()) {
|
||||
ctxBuilder.actionMode(entry.getKey(), entry.getValue());
|
||||
}
|
||||
if (request.getAlternativeInput() != null) {
|
||||
ctxBuilder.withInput(new SimpleInput.Result(new Payload.Simple(request.getAlternativeInput())));
|
||||
}
|
||||
if (request.isIgnoreCondition()) {
|
||||
ctxBuilder.withCondition(AlwaysCondition.RESULT_INSTANCE);
|
||||
}
|
||||
ctxBuilder.recordExecution(request.isRecordExecution());
|
||||
ManualExecutionContext.Builder ctxBuilder = ManualExecutionContext.builder(watch, knownWatch,
|
||||
new ManualTriggerEvent(triggerEvent.jobName(), triggerEvent), executionService.defaultThrottlePeriod());
|
||||
|
||||
WatchRecord record = executionService.execute(ctxBuilder.build());
|
||||
XContentBuilder builder = XContentFactory.jsonBuilder();
|
||||
DateTime executionTime = new DateTime(clock.millis(), UTC);
|
||||
ctxBuilder.executionTime(executionTime);
|
||||
for (Map.Entry<String, ActionExecutionMode> entry : request.getActionModes().entrySet()) {
|
||||
ctxBuilder.actionMode(entry.getKey(), entry.getValue());
|
||||
}
|
||||
if (request.getAlternativeInput() != null) {
|
||||
ctxBuilder.withInput(new SimpleInput.Result(new Payload.Simple(request.getAlternativeInput())));
|
||||
}
|
||||
if (request.isIgnoreCondition()) {
|
||||
ctxBuilder.withCondition(AlwaysCondition.RESULT_INSTANCE);
|
||||
}
|
||||
ctxBuilder.recordExecution(request.isRecordExecution());
|
||||
|
||||
record.toXContent(builder, WatcherParams.builder().hideSecrets(true).debug(request.isDebug()).build());
|
||||
return new ExecuteWatchResponse(record.id().value(), builder.bytes(), XContentType.JSON);
|
||||
WatchRecord record = executionService.execute(ctxBuilder.build());
|
||||
XContentBuilder builder = XContentFactory.jsonBuilder();
|
||||
|
||||
record.toXContent(builder, WatcherParams.builder().hideSecrets(true).debug(request.isDebug()).build());
|
||||
listener.onResponse(new ExecuteWatchResponse(record.id().value(), builder.bytes(), XContentType.JSON));
|
||||
} catch (IOException e) {
|
||||
listener.onFailure(e);
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
protected ClusterBlockException checkBlock(ExecuteWatchRequest request, ClusterState state) {
|
||||
return state.blocks().indexBlockedException(ClusterBlockLevel.WRITE, Watch.INDEX);
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
|
|
@ -5,23 +5,16 @@
|
|||
*/
|
||||
package org.elasticsearch.xpack.watcher.transport.actions.get;
|
||||
|
||||
import org.elasticsearch.ElasticsearchException;
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.action.support.ActionFilters;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
import org.elasticsearch.cluster.block.ClusterBlockException;
|
||||
import org.elasticsearch.cluster.block.ClusterBlockLevel;
|
||||
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
|
||||
import org.elasticsearch.cluster.service.ClusterService;
|
||||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.common.xcontent.XContentType;
|
||||
import org.elasticsearch.license.LicenseUtils;
|
||||
import org.elasticsearch.license.XPackLicenseState;
|
||||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
import org.elasticsearch.transport.TransportService;
|
||||
import org.elasticsearch.xpack.XPackPlugin;
|
||||
import org.elasticsearch.xpack.watcher.support.init.proxy.WatcherClientProxy;
|
||||
import org.elasticsearch.xpack.watcher.support.xcontent.WatcherParams;
|
||||
import org.elasticsearch.xpack.watcher.transport.actions.WatcherTransportAction;
|
||||
|
@ -40,35 +33,18 @@ public class TransportGetWatchAction extends WatcherTransportAction<GetWatchRequ
|
|||
private final WatcherClientProxy client;
|
||||
|
||||
@Inject
|
||||
public TransportGetWatchAction(Settings settings, TransportService transportService, ClusterService clusterService,
|
||||
ThreadPool threadPool, ActionFilters actionFilters,
|
||||
public TransportGetWatchAction(Settings settings, TransportService transportService, ThreadPool threadPool, ActionFilters actionFilters,
|
||||
IndexNameExpressionResolver indexNameExpressionResolver, XPackLicenseState licenseState,
|
||||
Watch.Parser parser, Clock clock, WatcherClientProxy client) {
|
||||
super(settings, GetWatchAction.NAME, transportService, clusterService, threadPool, actionFilters,
|
||||
indexNameExpressionResolver, licenseState, GetWatchRequest::new);
|
||||
super(settings, GetWatchAction.NAME, transportService, threadPool, actionFilters, indexNameExpressionResolver,
|
||||
licenseState, GetWatchRequest::new);
|
||||
this.parser = parser;
|
||||
this.clock = clock;
|
||||
this.client = client;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected String executor() {
|
||||
return ThreadPool.Names.MANAGEMENT;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected GetWatchResponse newResponse() {
|
||||
return new GetWatchResponse();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void masterOperation(GetWatchRequest request, ClusterState state, ActionListener<GetWatchResponse> listener) throws
|
||||
ElasticsearchException {
|
||||
if (licenseState.isWatcherAllowed() == false) {
|
||||
listener.onFailure(LicenseUtils.newComplianceException(XPackPlugin.WATCHER));
|
||||
return;
|
||||
}
|
||||
|
||||
protected void doExecute(GetWatchRequest request, ActionListener<GetWatchResponse> listener) {
|
||||
client.getWatch(request.getId(), ActionListener.wrap(getResponse -> {
|
||||
if (getResponse.isExists()) {
|
||||
try (XContentBuilder builder = jsonBuilder()) {
|
||||
|
@ -90,9 +66,4 @@ public class TransportGetWatchAction extends WatcherTransportAction<GetWatchRequ
|
|||
}
|
||||
}, listener::onFailure));
|
||||
}
|
||||
|
||||
@Override
|
||||
protected ClusterBlockException checkBlock(GetWatchRequest request, ClusterState state) {
|
||||
return state.blocks().indexBlockedException(ClusterBlockLevel.READ, Watch.INDEX);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -5,17 +5,12 @@
|
|||
*/
|
||||
package org.elasticsearch.xpack.watcher.transport.actions.put;
|
||||
|
||||
import org.elasticsearch.ElasticsearchException;
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.action.DocWriteResponse;
|
||||
import org.elasticsearch.action.index.IndexRequest;
|
||||
import org.elasticsearch.action.support.ActionFilters;
|
||||
import org.elasticsearch.action.support.WriteRequest;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
import org.elasticsearch.cluster.block.ClusterBlockException;
|
||||
import org.elasticsearch.cluster.block.ClusterBlockLevel;
|
||||
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
|
||||
import org.elasticsearch.cluster.service.ClusterService;
|
||||
import org.elasticsearch.common.bytes.BytesReference;
|
||||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
|
@ -27,7 +22,6 @@ import org.elasticsearch.threadpool.ThreadPool;
|
|||
import org.elasticsearch.transport.TransportService;
|
||||
import org.elasticsearch.xpack.security.InternalClient;
|
||||
import org.elasticsearch.xpack.watcher.transport.actions.WatcherTransportAction;
|
||||
import org.elasticsearch.xpack.watcher.trigger.TriggerService;
|
||||
import org.elasticsearch.xpack.watcher.watch.Payload;
|
||||
import org.elasticsearch.xpack.watcher.watch.Watch;
|
||||
import org.joda.time.DateTime;
|
||||
|
@ -41,37 +35,22 @@ import static org.joda.time.DateTimeZone.UTC;
|
|||
public class TransportPutWatchAction extends WatcherTransportAction<PutWatchRequest, PutWatchResponse> {
|
||||
|
||||
private final Clock clock;
|
||||
private final TriggerService triggerService;
|
||||
private final Watch.Parser parser;
|
||||
private final InternalClient client;
|
||||
|
||||
@Inject
|
||||
public TransportPutWatchAction(Settings settings, TransportService transportService, ClusterService clusterService,
|
||||
ThreadPool threadPool, ActionFilters actionFilters,
|
||||
public TransportPutWatchAction(Settings settings, TransportService transportService, ThreadPool threadPool, ActionFilters actionFilters,
|
||||
IndexNameExpressionResolver indexNameExpressionResolver, Clock clock, XPackLicenseState licenseState,
|
||||
TriggerService triggerService, Watch.Parser parser, InternalClient client) {
|
||||
super(settings, PutWatchAction.NAME, transportService, clusterService, threadPool, actionFilters,
|
||||
indexNameExpressionResolver, licenseState, PutWatchRequest::new);
|
||||
Watch.Parser parser, InternalClient client) {
|
||||
super(settings, PutWatchAction.NAME, transportService, threadPool, actionFilters, indexNameExpressionResolver,
|
||||
licenseState, PutWatchRequest::new);
|
||||
this.clock = clock;
|
||||
this.triggerService = triggerService;
|
||||
this.parser = parser;
|
||||
this.client = client;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected String executor() {
|
||||
return ThreadPool.Names.MANAGEMENT;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected PutWatchResponse newResponse() {
|
||||
return new PutWatchResponse();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void masterOperation(PutWatchRequest request, ClusterState state, ActionListener<PutWatchResponse> listener) throws
|
||||
ElasticsearchException {
|
||||
|
||||
protected void doExecute(PutWatchRequest request, ActionListener<PutWatchResponse> listener) {
|
||||
try {
|
||||
DateTime now = new DateTime(clock.millis(), UTC);
|
||||
Watch watch = parser.parseWithSecrets(request.getId(), false, request.getSource(), now, request.xContentType());
|
||||
|
@ -88,11 +67,6 @@ public class TransportPutWatchAction extends WatcherTransportAction<PutWatchRequ
|
|||
|
||||
client.index(indexRequest, ActionListener.wrap(indexResponse -> {
|
||||
boolean created = indexResponse.getResult() == DocWriteResponse.Result.CREATED;
|
||||
if (request.isActive()) {
|
||||
triggerService.add(watch);
|
||||
} else {
|
||||
triggerService.remove(request.getId());
|
||||
}
|
||||
listener.onResponse(new PutWatchResponse(indexResponse.getId(), indexResponse.getVersion(), created));
|
||||
}, listener::onFailure));
|
||||
}
|
||||
|
@ -100,10 +74,4 @@ public class TransportPutWatchAction extends WatcherTransportAction<PutWatchRequ
|
|||
listener.onFailure(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected ClusterBlockException checkBlock(PutWatchRequest request, ClusterState state) {
|
||||
return state.blocks().indexBlockedException(ClusterBlockLevel.WRITE, Watch.INDEX);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -5,41 +5,49 @@
|
|||
*/
|
||||
package org.elasticsearch.xpack.watcher.transport.actions.service;
|
||||
|
||||
import org.elasticsearch.ElasticsearchException;
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.action.support.ActionFilters;
|
||||
import org.elasticsearch.cluster.service.ClusterService;
|
||||
import org.elasticsearch.action.support.master.AcknowledgedRequest;
|
||||
import org.elasticsearch.action.support.master.TransportMasterNodeAction;
|
||||
import org.elasticsearch.cluster.AckedClusterStateUpdateTask;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
import org.elasticsearch.cluster.ack.AckedRequest;
|
||||
import org.elasticsearch.cluster.block.ClusterBlockException;
|
||||
import org.elasticsearch.cluster.block.ClusterBlockLevel;
|
||||
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
|
||||
import org.elasticsearch.cluster.metadata.MetaData;
|
||||
import org.elasticsearch.cluster.service.ClusterService;
|
||||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.license.XPackLicenseState;
|
||||
import org.elasticsearch.common.unit.TimeValue;
|
||||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
import org.elasticsearch.transport.TransportService;
|
||||
import org.elasticsearch.xpack.watcher.WatcherLifeCycleService;
|
||||
import org.elasticsearch.xpack.watcher.transport.actions.WatcherTransportAction;
|
||||
import org.elasticsearch.xpack.watcher.WatcherMetaData;
|
||||
|
||||
public class TransportWatcherServiceAction extends WatcherTransportAction<WatcherServiceRequest, WatcherServiceResponse> {
|
||||
public class TransportWatcherServiceAction extends TransportMasterNodeAction<WatcherServiceRequest, WatcherServiceResponse> {
|
||||
|
||||
private final WatcherLifeCycleService lifeCycleService;
|
||||
private AckedRequest ackedRequest = new AckedRequest() {
|
||||
@Override
|
||||
public TimeValue ackTimeout() {
|
||||
return AcknowledgedRequest.DEFAULT_ACK_TIMEOUT;
|
||||
}
|
||||
|
||||
@Override
|
||||
public TimeValue masterNodeTimeout() {
|
||||
return AcknowledgedRequest.DEFAULT_ACK_TIMEOUT;
|
||||
}
|
||||
};
|
||||
|
||||
@Inject
|
||||
public TransportWatcherServiceAction(Settings settings, TransportService transportService, ClusterService clusterService,
|
||||
ThreadPool threadPool, ActionFilters actionFilters,
|
||||
IndexNameExpressionResolver indexNameExpressionResolver,
|
||||
WatcherLifeCycleService lifeCycleService, XPackLicenseState licenseState) {
|
||||
IndexNameExpressionResolver indexNameExpressionResolver) {
|
||||
super(settings, WatcherServiceAction.NAME, transportService, clusterService, threadPool, actionFilters,
|
||||
indexNameExpressionResolver, licenseState, WatcherServiceRequest::new);
|
||||
this.lifeCycleService = lifeCycleService;
|
||||
indexNameExpressionResolver, WatcherServiceRequest::new);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected String executor() {
|
||||
// Ideally we should use SAME TP here, because we should always be able to stop Watcher even if management TP has been exhausted,
|
||||
// but we can't use SAME TP here, because certain parts of the start process can't be executed on a transport thread.
|
||||
// ( put template, or anything client related with #actionGet() )
|
||||
return ThreadPool.Names.MANAGEMENT;
|
||||
}
|
||||
|
||||
|
@ -49,24 +57,44 @@ public class TransportWatcherServiceAction extends WatcherTransportAction<Watche
|
|||
}
|
||||
|
||||
@Override
|
||||
protected void masterOperation(WatcherServiceRequest request, ClusterState state, ActionListener<WatcherServiceResponse> listener)
|
||||
throws ElasticsearchException {
|
||||
protected void masterOperation(WatcherServiceRequest request, ClusterState state,
|
||||
ActionListener<WatcherServiceResponse> listener) throws Exception {
|
||||
switch (request.getCommand()) {
|
||||
case START:
|
||||
lifeCycleService.start();
|
||||
break;
|
||||
case STOP:
|
||||
lifeCycleService.stop();
|
||||
setWatcherMetaDataAndWait(true, listener);
|
||||
break;
|
||||
case RESTART:
|
||||
lifeCycleService.stop();
|
||||
lifeCycleService.start();
|
||||
case START:
|
||||
setWatcherMetaDataAndWait(false, listener);
|
||||
break;
|
||||
default:
|
||||
listener.onFailure(new IllegalArgumentException("Command [" + request.getCommand() + "] is undefined"));
|
||||
return;
|
||||
}
|
||||
listener.onResponse(new WatcherServiceResponse(true));
|
||||
}
|
||||
|
||||
private void setWatcherMetaDataAndWait(boolean manuallyStopped, final ActionListener
|
||||
<WatcherServiceResponse> listener) {
|
||||
String source = manuallyStopped ? "update_watcher_manually_stopped" : "update_watcher_manually_started";
|
||||
|
||||
clusterService.submitStateUpdateTask(source,
|
||||
new AckedClusterStateUpdateTask<Boolean>(ackedRequest,
|
||||
ActionListener.wrap(b -> listener.onResponse(new WatcherServiceResponse(true)), listener::onFailure)) {
|
||||
|
||||
@Override
|
||||
protected Boolean newResponse(boolean result) {
|
||||
return result;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ClusterState execute(ClusterState clusterState) throws Exception {
|
||||
ClusterState.Builder builder = new ClusterState.Builder(clusterState);
|
||||
builder.metaData(MetaData.builder(clusterState.getMetaData())
|
||||
.putCustom(WatcherMetaData.TYPE, new WatcherMetaData(manuallyStopped)));
|
||||
return builder.build();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onFailure(String source, Exception throwable) {
|
||||
listener.onFailure(throwable);
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -16,7 +16,7 @@ import java.util.Locale;
|
|||
|
||||
public class WatcherServiceRequest extends MasterNodeRequest<WatcherServiceRequest> {
|
||||
|
||||
enum Command { START, STOP, RESTART }
|
||||
enum Command { START, STOP }
|
||||
|
||||
private Command command;
|
||||
|
||||
|
@ -39,14 +39,6 @@ public class WatcherServiceRequest extends MasterNodeRequest<WatcherServiceReque
|
|||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Starts and stops the watcher.
|
||||
*/
|
||||
public WatcherServiceRequest restart() {
|
||||
command = Command.RESTART;
|
||||
return this;
|
||||
}
|
||||
|
||||
Command getCommand() {
|
||||
return command;
|
||||
}
|
||||
|
|
|
@ -16,7 +16,7 @@ public class WatcherServiceRequestBuilder extends MasterNodeOperationRequestBuil
|
|||
}
|
||||
|
||||
/**
|
||||
* Starts the watcher if not already started.
|
||||
* Starts watcher if not already started.
|
||||
*/
|
||||
public WatcherServiceRequestBuilder start() {
|
||||
request.start();
|
||||
|
@ -24,18 +24,10 @@ public class WatcherServiceRequestBuilder extends MasterNodeOperationRequestBuil
|
|||
}
|
||||
|
||||
/**
|
||||
* Stops the watcher if not already stopped.
|
||||
* Stops watcher if not already stopped.
|
||||
*/
|
||||
public WatcherServiceRequestBuilder stop() {
|
||||
request.stop();
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Starts and stops the watcher.
|
||||
*/
|
||||
public WatcherServiceRequestBuilder restart() {
|
||||
request.restart();
|
||||
return this;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -11,12 +11,9 @@ import org.elasticsearch.common.io.stream.StreamOutput;
|
|||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
* Empty response, so if it returns, it means all is fine.
|
||||
*/
|
||||
public class WatcherServiceResponse extends AcknowledgedResponse {
|
||||
|
||||
WatcherServiceResponse() {
|
||||
public WatcherServiceResponse() {
|
||||
}
|
||||
|
||||
public WatcherServiceResponse(boolean acknowledged) {
|
||||
|
|
|
@ -5,72 +5,70 @@
|
|||
*/
|
||||
package org.elasticsearch.xpack.watcher.transport.actions.stats;
|
||||
|
||||
import org.elasticsearch.ElasticsearchException;
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.action.search.SearchRequest;
|
||||
import org.elasticsearch.action.FailedNodeException;
|
||||
import org.elasticsearch.action.support.ActionFilters;
|
||||
import org.elasticsearch.client.Requests;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
import org.elasticsearch.cluster.block.ClusterBlockException;
|
||||
import org.elasticsearch.cluster.block.ClusterBlockLevel;
|
||||
import org.elasticsearch.action.support.nodes.TransportNodesAction;
|
||||
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
|
||||
import org.elasticsearch.cluster.service.ClusterService;
|
||||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.license.XPackLicenseState;
|
||||
import org.elasticsearch.search.builder.SearchSourceBuilder;
|
||||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
import org.elasticsearch.transport.TransportService;
|
||||
import org.elasticsearch.xpack.security.InternalClient;
|
||||
import org.elasticsearch.xpack.watcher.WatcherLifeCycleService;
|
||||
import org.elasticsearch.xpack.watcher.WatcherService;
|
||||
import org.elasticsearch.xpack.watcher.execution.ExecutionService;
|
||||
import org.elasticsearch.xpack.watcher.transport.actions.WatcherTransportAction;
|
||||
import org.elasticsearch.xpack.watcher.watch.Watch;
|
||||
import org.elasticsearch.xpack.watcher.trigger.TriggerService;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* Performs the stats operation.
|
||||
*/
|
||||
public class TransportWatcherStatsAction extends WatcherTransportAction<WatcherStatsRequest, WatcherStatsResponse> {
|
||||
public class TransportWatcherStatsAction extends TransportNodesAction<WatcherStatsRequest, WatcherStatsResponse,
|
||||
WatcherStatsRequest.Node, WatcherStatsResponse.Node> {
|
||||
|
||||
private final WatcherService watcherService;
|
||||
private final ExecutionService executionService;
|
||||
private final TriggerService triggerService;
|
||||
private final WatcherLifeCycleService lifeCycleService;
|
||||
private final InternalClient client;
|
||||
|
||||
@Inject
|
||||
public TransportWatcherStatsAction(Settings settings, TransportService transportService, ClusterService clusterService,
|
||||
ThreadPool threadPool, ActionFilters actionFilters,
|
||||
IndexNameExpressionResolver indexNameExpressionResolver, WatcherService watcherService,
|
||||
ExecutionService executionService, XPackLicenseState licenseState,
|
||||
WatcherLifeCycleService lifeCycleService, InternalClient client) {
|
||||
super(settings, WatcherStatsAction.NAME, transportService, clusterService, threadPool, actionFilters, indexNameExpressionResolver,
|
||||
licenseState, WatcherStatsRequest::new);
|
||||
ExecutionService executionService, TriggerService triggerService,
|
||||
WatcherLifeCycleService lifeCycleService) {
|
||||
super(settings, WatcherStatsAction.NAME, threadPool, clusterService, transportService, actionFilters, indexNameExpressionResolver,
|
||||
WatcherStatsRequest::new, WatcherStatsRequest.Node::new, ThreadPool.Names.MANAGEMENT,
|
||||
WatcherStatsResponse.Node.class);
|
||||
this.watcherService = watcherService;
|
||||
this.executionService = executionService;
|
||||
this.triggerService = triggerService;
|
||||
this.lifeCycleService = lifeCycleService;
|
||||
this.client = client;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected String executor() {
|
||||
return ThreadPool.Names.SAME;
|
||||
protected WatcherStatsResponse newResponse(WatcherStatsRequest request, List<WatcherStatsResponse.Node> nodes,
|
||||
List<FailedNodeException> failures) {
|
||||
return new WatcherStatsResponse(clusterService.getClusterName(), lifeCycleService.watcherMetaData(), nodes, failures);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected WatcherStatsResponse newResponse() {
|
||||
return new WatcherStatsResponse();
|
||||
protected WatcherStatsRequest.Node newNodeRequest(String nodeId, WatcherStatsRequest request) {
|
||||
return new WatcherStatsRequest.Node(request, nodeId);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void masterOperation(WatcherStatsRequest request, ClusterState state, ActionListener<WatcherStatsResponse> listener) throws
|
||||
ElasticsearchException {
|
||||
protected WatcherStatsResponse.Node newNodeResponse() {
|
||||
return new WatcherStatsResponse.Node();
|
||||
}
|
||||
|
||||
WatcherStatsResponse statsResponse = new WatcherStatsResponse();
|
||||
@Override
|
||||
protected WatcherStatsResponse.Node nodeOperation(WatcherStatsRequest.Node request) {
|
||||
WatcherStatsResponse.Node statsResponse = new WatcherStatsResponse.Node(clusterService.localNode());
|
||||
statsResponse.setWatcherState(watcherService.state());
|
||||
statsResponse.setThreadPoolQueueSize(executionService.executionThreadPoolQueueSize());
|
||||
statsResponse.setThreadPoolMaxSize(executionService.executionThreadPoolMaxSize());
|
||||
statsResponse.setWatcherMetaData(lifeCycleService.watcherMetaData());
|
||||
if (request.includeCurrentWatches()) {
|
||||
statsResponse.setSnapshots(executionService.currentExecutions());
|
||||
}
|
||||
|
@ -78,17 +76,13 @@ public class TransportWatcherStatsAction extends WatcherTransportAction<WatcherS
|
|||
statsResponse.setQueuedWatches(executionService.queuedWatches());
|
||||
}
|
||||
|
||||
SearchRequest searchRequest =
|
||||
Requests.searchRequest(Watch.INDEX).types(Watch.DOC_TYPE).source(new SearchSourceBuilder().size(0));
|
||||
client.search(searchRequest, ActionListener.wrap(searchResponse -> {
|
||||
statsResponse.setWatchesCount(searchResponse.getHits().getTotalHits());
|
||||
listener.onResponse(statsResponse);
|
||||
},
|
||||
e -> listener.onResponse(statsResponse)));
|
||||
statsResponse.setWatchesCount(triggerService.count());
|
||||
return statsResponse;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected ClusterBlockException checkBlock(WatcherStatsRequest request, ClusterState state) {
|
||||
return state.blocks().globalBlockedException(ClusterBlockLevel.METADATA_READ);
|
||||
protected boolean accumulateExceptions() {
|
||||
return false;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -6,16 +6,18 @@
|
|||
package org.elasticsearch.xpack.watcher.transport.actions.stats;
|
||||
|
||||
import org.elasticsearch.action.ActionRequestValidationException;
|
||||
import org.elasticsearch.action.support.master.MasterNodeReadRequest;
|
||||
import org.elasticsearch.action.support.nodes.BaseNodeRequest;
|
||||
import org.elasticsearch.action.support.nodes.BaseNodesRequest;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.xpack.watcher.transport.actions.service.WatcherServiceRequest;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
* The Request to get the watcher stats
|
||||
*/
|
||||
public class WatcherStatsRequest extends MasterNodeReadRequest<WatcherStatsRequest> {
|
||||
public class WatcherStatsRequest extends BaseNodesRequest<WatcherStatsRequest> {
|
||||
|
||||
private boolean includeCurrentWatches;
|
||||
private boolean includeQueuedWatches;
|
||||
|
@ -62,4 +64,40 @@ public class WatcherStatsRequest extends MasterNodeReadRequest<WatcherStatsReque
|
|||
public String toString() {
|
||||
return "watcher_stats";
|
||||
}
|
||||
}
|
||||
|
||||
public static class Node extends BaseNodeRequest {
|
||||
|
||||
private boolean includeCurrentWatches;
|
||||
private boolean includeQueuedWatches;
|
||||
|
||||
public Node() {}
|
||||
|
||||
public Node(WatcherStatsRequest request, String nodeId) {
|
||||
super(nodeId);
|
||||
includeCurrentWatches = request.includeCurrentWatches();
|
||||
includeQueuedWatches = request.includeQueuedWatches();
|
||||
}
|
||||
|
||||
public boolean includeCurrentWatches() {
|
||||
return includeCurrentWatches;
|
||||
}
|
||||
|
||||
public boolean includeQueuedWatches() {
|
||||
return includeQueuedWatches;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void readFrom(StreamInput in) throws IOException {
|
||||
super.readFrom(in);
|
||||
includeCurrentWatches = in.readBoolean();
|
||||
includeQueuedWatches = in.readBoolean();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeTo(StreamOutput out) throws IOException {
|
||||
super.writeTo(out);
|
||||
out.writeBoolean(includeCurrentWatches);
|
||||
out.writeBoolean(includeQueuedWatches);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -5,13 +5,13 @@
|
|||
*/
|
||||
package org.elasticsearch.xpack.watcher.transport.actions.stats;
|
||||
|
||||
import org.elasticsearch.action.support.master.MasterNodeReadOperationRequestBuilder;
|
||||
import org.elasticsearch.action.support.nodes.NodesOperationRequestBuilder;
|
||||
import org.elasticsearch.client.ElasticsearchClient;
|
||||
|
||||
/**
|
||||
* Watcher stats request builder.
|
||||
*/
|
||||
public class WatcherStatsRequestBuilder extends MasterNodeReadOperationRequestBuilder<WatcherStatsRequest, WatcherStatsResponse,
|
||||
public class WatcherStatsRequestBuilder extends NodesOperationRequestBuilder<WatcherStatsRequest, WatcherStatsResponse,
|
||||
WatcherStatsRequestBuilder> {
|
||||
|
||||
public WatcherStatsRequestBuilder(ElasticsearchClient client) {
|
||||
|
|
|
@ -5,7 +5,12 @@
|
|||
*/
|
||||
package org.elasticsearch.xpack.watcher.transport.actions.stats;
|
||||
|
||||
import org.elasticsearch.action.ActionResponse;
|
||||
import org.elasticsearch.Version;
|
||||
import org.elasticsearch.action.FailedNodeException;
|
||||
import org.elasticsearch.action.support.nodes.BaseNodeResponse;
|
||||
import org.elasticsearch.action.support.nodes.BaseNodesResponse;
|
||||
import org.elasticsearch.cluster.ClusterName;
|
||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||
import org.elasticsearch.common.Nullable;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
|
@ -17,175 +22,271 @@ import org.elasticsearch.xpack.watcher.execution.QueuedWatch;
|
|||
import org.elasticsearch.xpack.watcher.execution.WatchExecutionSnapshot;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Locale;
|
||||
|
||||
public class WatcherStatsResponse extends ActionResponse implements ToXContentObject {
|
||||
public class WatcherStatsResponse extends BaseNodesResponse<WatcherStatsResponse.Node>
|
||||
implements ToXContentObject {
|
||||
|
||||
private long watchesCount;
|
||||
private WatcherState watcherState;
|
||||
private long threadPoolQueueSize;
|
||||
private long threadPoolMaxSize;
|
||||
private WatcherMetaData watcherMetaData;
|
||||
|
||||
private List<WatchExecutionSnapshot> snapshots;
|
||||
private List<QueuedWatch> queuedWatches;
|
||||
|
||||
WatcherStatsResponse() {
|
||||
public WatcherStatsResponse() {
|
||||
}
|
||||
|
||||
/**
|
||||
* @return The current execution thread pool queue size
|
||||
*/
|
||||
public long getThreadPoolQueueSize() {
|
||||
return threadPoolQueueSize;
|
||||
}
|
||||
|
||||
void setThreadPoolQueueSize(long threadPoolQueueSize) {
|
||||
this.threadPoolQueueSize = threadPoolQueueSize;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return The max number of threads in the execution thread pool
|
||||
*/
|
||||
public long getThreadPoolMaxSize() {
|
||||
return threadPoolMaxSize;
|
||||
}
|
||||
|
||||
void setThreadPoolMaxSize(long threadPoolMaxSize) {
|
||||
this.threadPoolMaxSize = threadPoolMaxSize;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return The number of watches currently registered in the system
|
||||
*/
|
||||
public long getWatchesCount() {
|
||||
return watchesCount;
|
||||
}
|
||||
|
||||
void setWatchesCount(long watchesCount) {
|
||||
this.watchesCount = watchesCount;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return The state of the watch service.
|
||||
*/
|
||||
public WatcherState getWatcherState() {
|
||||
return watcherState;
|
||||
}
|
||||
|
||||
void setWatcherState(WatcherState watcherServiceState) {
|
||||
this.watcherState = watcherServiceState;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
public List<WatchExecutionSnapshot> getSnapshots() {
|
||||
return snapshots;
|
||||
}
|
||||
|
||||
void setSnapshots(List<WatchExecutionSnapshot> snapshots) {
|
||||
this.snapshots = snapshots;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
public List<QueuedWatch> getQueuedWatches() {
|
||||
return queuedWatches;
|
||||
}
|
||||
|
||||
public void setQueuedWatches(List<QueuedWatch> queuedWatches) {
|
||||
this.queuedWatches = queuedWatches;
|
||||
}
|
||||
|
||||
public WatcherMetaData getWatcherMetaData() {
|
||||
return watcherMetaData;
|
||||
}
|
||||
|
||||
public void setWatcherMetaData(WatcherMetaData watcherMetaData) {
|
||||
public WatcherStatsResponse(ClusterName clusterName, WatcherMetaData watcherMetaData,
|
||||
List<Node> nodes, List<FailedNodeException> failures) {
|
||||
super(clusterName, nodes, failures);
|
||||
this.watcherMetaData = watcherMetaData;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void readFrom(StreamInput in) throws IOException {
|
||||
super.readFrom(in);
|
||||
watchesCount = in.readLong();
|
||||
threadPoolQueueSize = in.readLong();
|
||||
threadPoolMaxSize = in.readLong();
|
||||
watcherState = WatcherState.fromId(in.readByte());
|
||||
public void writeTo(StreamOutput out) throws IOException {
|
||||
// if (out.getVersion().after(Version.V_6_0_0_alpha1_UNRELEASED)) {
|
||||
super.writeTo(out);
|
||||
out.writeBoolean(watcherMetaData.manuallyStopped());
|
||||
/*
|
||||
} else {
|
||||
// BWC layer for older versions, this is not considered exact
|
||||
// this mimics the behaviour of 5.x
|
||||
out.writeLong(getNodes().stream().mapToLong(Node::getWatchesCount).sum());
|
||||
out.writeLong(getNodes().stream().mapToLong(Node::getThreadPoolQueueSize).sum());
|
||||
out.writeLong(getNodes().stream().mapToLong(Node::getThreadPoolMaxSize).sum());
|
||||
// byte, watcher state, cannot be exact, just pick the first
|
||||
out.writeByte(getNodes().get(0).getWatcherState().getId());
|
||||
|
||||
if (in.readBoolean()) {
|
||||
int size = in.readVInt();
|
||||
snapshots = new ArrayList<>(size);
|
||||
for (int i = 0; i < size; i++) {
|
||||
snapshots.add(new WatchExecutionSnapshot(in));
|
||||
out.writeString(Version.CURRENT.toString()); // version
|
||||
out.writeString(XPackBuild.CURRENT.shortHash()); // hash
|
||||
out.writeString(XPackBuild.CURRENT.shortHash()); // short hash
|
||||
out.writeString(XPackBuild.CURRENT.date()); // date
|
||||
|
||||
List<WatchExecutionSnapshot> snapshots = getNodes().stream().map(Node::getSnapshots)
|
||||
.flatMap(List::stream)
|
||||
.collect(Collectors.toList());
|
||||
if (snapshots != null) {
|
||||
out.writeBoolean(true);
|
||||
out.writeVInt(snapshots.size());
|
||||
for (WatchExecutionSnapshot snapshot : snapshots) {
|
||||
snapshot.writeTo(out);
|
||||
}
|
||||
} else {
|
||||
out.writeBoolean(false);
|
||||
}
|
||||
}
|
||||
if (in.readBoolean()) {
|
||||
int size = in.readVInt();
|
||||
queuedWatches = new ArrayList<>(size);
|
||||
for (int i = 0; i < size; i++) {
|
||||
queuedWatches.add(new QueuedWatch(in));
|
||||
|
||||
List<QueuedWatch> queuedWatches = getNodes().stream().map(Node::getQueuedWatches)
|
||||
.flatMap(List::stream)
|
||||
.collect(Collectors.toList());
|
||||
if (queuedWatches != null) {
|
||||
out.writeBoolean(true);
|
||||
out.writeVInt(queuedWatches.size());
|
||||
for (QueuedWatch pending : queuedWatches) {
|
||||
pending.writeTo(out);
|
||||
}
|
||||
} else {
|
||||
out.writeBoolean(false);
|
||||
}
|
||||
|
||||
watcherMetaData.writeTo(out);
|
||||
}
|
||||
watcherMetaData = new WatcherMetaData(in);
|
||||
*/
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeTo(StreamOutput out) throws IOException {
|
||||
super.writeTo(out);
|
||||
out.writeLong(watchesCount);
|
||||
out.writeLong(threadPoolQueueSize);
|
||||
out.writeLong(threadPoolMaxSize);
|
||||
out.writeByte(watcherState.getId());
|
||||
public void readFrom(StreamInput in) throws IOException {
|
||||
if (in.getVersion().onOrAfter(Version.V_6_0_0_alpha1_UNRELEASED)) {
|
||||
super.readFrom(in);
|
||||
watcherMetaData = new WatcherMetaData(in.readBoolean());
|
||||
} else {
|
||||
// TODO what to do here? create another BWC helping stuff here...
|
||||
}
|
||||
}
|
||||
|
||||
if (snapshots != null) {
|
||||
out.writeBoolean(true);
|
||||
out.writeVInt(snapshots.size());
|
||||
for (WatchExecutionSnapshot snapshot : snapshots) {
|
||||
snapshot.writeTo(out);
|
||||
}
|
||||
} else {
|
||||
out.writeBoolean(false);
|
||||
}
|
||||
if (queuedWatches != null) {
|
||||
out.writeBoolean(true);
|
||||
out.writeVInt(queuedWatches.size());
|
||||
for (QueuedWatch pending : this.queuedWatches) {
|
||||
pending.writeTo(out);
|
||||
}
|
||||
} else {
|
||||
out.writeBoolean(false);
|
||||
}
|
||||
watcherMetaData.writeTo(out);
|
||||
@Override
|
||||
protected List<Node> readNodesFrom(StreamInput in) throws IOException {
|
||||
return in.readList(Node::readNodeResponse);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void writeNodesTo(StreamOutput out, List<Node> nodes) throws IOException {
|
||||
out.writeStreamableList(nodes);
|
||||
}
|
||||
|
||||
@Override
|
||||
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
|
||||
builder.startObject();
|
||||
builder.field("watcher_state", watcherState.toString().toLowerCase(Locale.ROOT));
|
||||
builder.field("watch_count", watchesCount);
|
||||
builder.startObject("execution_thread_pool");
|
||||
builder.field("queue_size", threadPoolQueueSize);
|
||||
builder.field("max_size", threadPoolMaxSize);
|
||||
builder.endObject();
|
||||
|
||||
if (snapshots != null) {
|
||||
builder.startArray("current_watches");
|
||||
for (WatchExecutionSnapshot snapshot : snapshots) {
|
||||
snapshot.toXContent(builder, params);
|
||||
}
|
||||
builder.endArray();
|
||||
}
|
||||
if (queuedWatches != null) {
|
||||
builder.startArray("queued_watches");
|
||||
for (QueuedWatch queuedWatch : queuedWatches) {
|
||||
queuedWatch.toXContent(builder, params);
|
||||
}
|
||||
builder.endArray();
|
||||
}
|
||||
watcherMetaData.toXContent(builder, params);
|
||||
builder.startArray("stats");
|
||||
for (Node node : getNodes()) {
|
||||
node.toXContent(builder, params);
|
||||
}
|
||||
builder.endArray();
|
||||
|
||||
builder.endObject();
|
||||
return builder;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Sum all watches across all nodes to get a total count of watches in the cluster
|
||||
*
|
||||
* @return The sum of all watches being executed
|
||||
*/
|
||||
public long getWatchesCount() {
|
||||
return getNodes().stream().mapToLong(WatcherStatsResponse.Node::getWatchesCount).sum();
|
||||
}
|
||||
|
||||
public WatcherMetaData watcherMetaData() {
|
||||
return watcherMetaData;
|
||||
}
|
||||
|
||||
public static class Node extends BaseNodeResponse implements ToXContentObject {
|
||||
|
||||
private long watchesCount;
|
||||
private WatcherState watcherState;
|
||||
private long threadPoolQueueSize;
|
||||
private long threadPoolMaxSize;
|
||||
private List<WatchExecutionSnapshot> snapshots;
|
||||
private List<QueuedWatch> queuedWatches;
|
||||
|
||||
Node() {
|
||||
}
|
||||
|
||||
Node(DiscoveryNode node) {
|
||||
super(node);
|
||||
}
|
||||
|
||||
/**
|
||||
* @return The current execution thread pool queue size
|
||||
*/
|
||||
public long getThreadPoolQueueSize() {
|
||||
return threadPoolQueueSize;
|
||||
}
|
||||
|
||||
void setThreadPoolQueueSize(long threadPoolQueueSize) {
|
||||
this.threadPoolQueueSize = threadPoolQueueSize;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return The max number of threads in the execution thread pool
|
||||
*/
|
||||
public long getThreadPoolMaxSize() {
|
||||
return threadPoolMaxSize;
|
||||
}
|
||||
|
||||
void setThreadPoolMaxSize(long threadPoolMaxSize) {
|
||||
this.threadPoolMaxSize = threadPoolMaxSize;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return The number of watches currently registered in the system
|
||||
*/
|
||||
public long getWatchesCount() {
|
||||
return watchesCount;
|
||||
}
|
||||
|
||||
void setWatchesCount(long watchesCount) {
|
||||
this.watchesCount = watchesCount;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return The state of the watch service.
|
||||
*/
|
||||
public WatcherState getWatcherState() {
|
||||
return watcherState;
|
||||
}
|
||||
|
||||
void setWatcherState(WatcherState watcherServiceState) {
|
||||
this.watcherState = watcherServiceState;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
public List<WatchExecutionSnapshot> getSnapshots() {
|
||||
return snapshots;
|
||||
}
|
||||
|
||||
void setSnapshots(List<WatchExecutionSnapshot> snapshots) {
|
||||
this.snapshots = snapshots;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
public List<QueuedWatch> getQueuedWatches() {
|
||||
return queuedWatches;
|
||||
}
|
||||
|
||||
public void setQueuedWatches(List<QueuedWatch> queuedWatches) {
|
||||
this.queuedWatches = queuedWatches;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void readFrom(StreamInput in) throws IOException {
|
||||
super.readFrom(in);
|
||||
watchesCount = in.readLong();
|
||||
threadPoolQueueSize = in.readLong();
|
||||
threadPoolMaxSize = in.readLong();
|
||||
watcherState = WatcherState.fromId(in.readByte());
|
||||
|
||||
if (in.readBoolean()) {
|
||||
snapshots = in.readStreamableList(WatchExecutionSnapshot::new);
|
||||
}
|
||||
if (in.readBoolean()) {
|
||||
queuedWatches = in.readStreamableList(QueuedWatch::new);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeTo(StreamOutput out) throws IOException {
|
||||
super.writeTo(out);
|
||||
out.writeLong(watchesCount);
|
||||
out.writeLong(threadPoolQueueSize);
|
||||
out.writeLong(threadPoolMaxSize);
|
||||
out.writeByte(watcherState.getId());
|
||||
|
||||
if (snapshots != null) {
|
||||
out.writeBoolean(true);
|
||||
out.writeStreamableList(snapshots);
|
||||
} else {
|
||||
out.writeBoolean(false);
|
||||
}
|
||||
if (queuedWatches != null) {
|
||||
out.writeBoolean(true);
|
||||
out.writeStreamableList(queuedWatches);
|
||||
} else {
|
||||
out.writeBoolean(false);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public XContentBuilder toXContent(XContentBuilder builder, Params params)
|
||||
throws IOException {
|
||||
builder.startObject();
|
||||
builder.field("node_id", getNode().getId());
|
||||
builder.field("watcher_state", watcherState.toString().toLowerCase(Locale.ROOT));
|
||||
builder.field("watch_count", watchesCount);
|
||||
builder.startObject("execution_thread_pool");
|
||||
builder.field("queue_size", threadPoolQueueSize);
|
||||
builder.field("max_size", threadPoolMaxSize);
|
||||
builder.endObject();
|
||||
|
||||
if (snapshots != null) {
|
||||
builder.startArray("current_watches");
|
||||
for (WatchExecutionSnapshot snapshot : snapshots) {
|
||||
snapshot.toXContent(builder, params);
|
||||
}
|
||||
builder.endArray();
|
||||
}
|
||||
if (queuedWatches != null) {
|
||||
builder.startArray("queued_watches");
|
||||
for (QueuedWatch queuedWatch : queuedWatches) {
|
||||
queuedWatch.toXContent(builder, params);
|
||||
}
|
||||
builder.endArray();
|
||||
}
|
||||
builder.endObject();
|
||||
return builder;
|
||||
}
|
||||
|
||||
public static WatcherStatsResponse.Node readNodeResponse(StreamInput in)
|
||||
throws IOException {
|
||||
WatcherStatsResponse.Node node = new WatcherStatsResponse.Node();
|
||||
node.readFrom(in);
|
||||
return node;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -30,6 +30,16 @@ public interface TriggerEngine<T extends Trigger, E extends TriggerEvent> {
|
|||
|
||||
void add(Watch job);
|
||||
|
||||
/**
|
||||
* Get into a pause state, implies clearing out existing jobs
|
||||
*/
|
||||
void pauseExecution();
|
||||
|
||||
/**
|
||||
* Returns the number of active jobs currently in this trigger engine implementation
|
||||
*/
|
||||
int getJobCount();
|
||||
|
||||
/**
|
||||
* Removes the job associated with the given name from this trigger engine.
|
||||
*
|
||||
|
|
|
@ -50,6 +50,21 @@ public class TriggerService extends AbstractComponent {
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Stop execution/triggering of watches on this node, do not try to reload anything, just sit still
|
||||
*/
|
||||
public synchronized void pauseExecution() {
|
||||
engines.values().forEach(TriggerEngine::pauseExecution);
|
||||
}
|
||||
|
||||
/**
|
||||
* Count the total number of active jobs across all trigger engines
|
||||
* @return The total count of active jobs
|
||||
*/
|
||||
public long count() {
|
||||
return engines.values().stream().mapToInt(TriggerEngine::getJobCount).sum();
|
||||
}
|
||||
|
||||
/**
|
||||
* Adds the given job to the trigger service. If there is already a registered job in this service with the
|
||||
* same job ID, the newly added job will replace the old job (the old job will not be triggered anymore)
|
||||
|
|
|
@ -48,6 +48,15 @@ public class ManualTriggerEngine implements TriggerEngine<ManualTrigger, ManualT
|
|||
public void add(Watch job) {
|
||||
}
|
||||
|
||||
@Override
|
||||
public void pauseExecution() {
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getJobCount() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean remove(String jobId) {
|
||||
return false;
|
||||
|
|
|
@ -5,6 +5,8 @@
|
|||
*/
|
||||
package org.elasticsearch.xpack.watcher.trigger.schedule.engine;
|
||||
|
||||
import org.elasticsearch.common.settings.Setting;
|
||||
import org.elasticsearch.common.settings.Setting.Property;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.unit.TimeValue;
|
||||
import org.elasticsearch.xpack.watcher.trigger.TriggerEvent;
|
||||
|
@ -24,17 +26,21 @@ import java.util.Map;
|
|||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
|
||||
import static org.elasticsearch.common.settings.Setting.positiveTimeSetting;
|
||||
import static org.joda.time.DateTimeZone.UTC;
|
||||
|
||||
public class TickerScheduleTriggerEngine extends ScheduleTriggerEngine {
|
||||
|
||||
public static final Setting<TimeValue> TICKER_INTERVAL_SETTING =
|
||||
positiveTimeSetting("xpack.watcher.trigger.schedule.ticker.tick_interval", TimeValue.timeValueMillis(500), Property.NodeScope);
|
||||
|
||||
private final TimeValue tickInterval;
|
||||
private volatile Map<String, ActiveSchedule> schedules;
|
||||
private Ticker ticker;
|
||||
|
||||
public TickerScheduleTriggerEngine(Settings settings, ScheduleRegistry scheduleRegistry, Clock clock) {
|
||||
super(settings, scheduleRegistry, clock);
|
||||
this.tickInterval = settings.getAsTime("xpack.watcher.trigger.schedule.ticker.tick_interval", TimeValue.timeValueMillis(500));
|
||||
this.tickInterval = TICKER_INTERVAL_SETTING.get(settings);
|
||||
this.schedules = new ConcurrentHashMap<>();
|
||||
}
|
||||
|
||||
|
@ -55,6 +61,7 @@ public class TickerScheduleTriggerEngine extends ScheduleTriggerEngine {
|
|||
@Override
|
||||
public void stop() {
|
||||
ticker.close();
|
||||
pauseExecution();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -64,6 +71,16 @@ public class TickerScheduleTriggerEngine extends ScheduleTriggerEngine {
|
|||
schedules.put(job.id(), new ActiveSchedule(job.id(), trigger.getSchedule(), clock.millis()));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void pauseExecution() {
|
||||
schedules.clear();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getJobCount() {
|
||||
return schedules.size();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean remove(String jobId) {
|
||||
return schedules.remove(jobId) != null;
|
||||
|
@ -75,7 +92,7 @@ public class TickerScheduleTriggerEngine extends ScheduleTriggerEngine {
|
|||
for (ActiveSchedule schedule : schedules.values()) {
|
||||
long scheduledTime = schedule.check(triggeredTime);
|
||||
if (scheduledTime > 0) {
|
||||
logger.trace("triggered job [{}] at [{}] (scheduled time was [{}])", schedule.name,
|
||||
logger.debug("triggered job [{}] at [{}] (scheduled time was [{}])", schedule.name,
|
||||
new DateTime(triggeredTime, UTC), new DateTime(scheduledTime, UTC));
|
||||
events.add(new ScheduleTriggerEvent(schedule.name, new DateTime(triggeredTime, UTC),
|
||||
new DateTime(scheduledTime, UTC)));
|
||||
|
@ -85,7 +102,7 @@ public class TickerScheduleTriggerEngine extends ScheduleTriggerEngine {
|
|||
}
|
||||
}
|
||||
}
|
||||
if (events.size() > 0) {
|
||||
if (events.isEmpty() == false) {
|
||||
notifyListeners(events);
|
||||
}
|
||||
}
|
||||
|
@ -161,5 +178,4 @@ public class TickerScheduleTriggerEngine extends ScheduleTriggerEngine {
|
|||
logger.trace("ticker thread stopped");
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -44,6 +44,9 @@
|
|||
"watch_id": {
|
||||
"type": "keyword"
|
||||
},
|
||||
"node": {
|
||||
"type": "keyword"
|
||||
},
|
||||
"trigger_event": {
|
||||
"type": "object",
|
||||
"dynamic": true,
|
||||
|
|
|
@ -16,10 +16,14 @@ import org.elasticsearch.search.lookup.SearchLookup;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
import java.util.Locale;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import static org.hamcrest.MatcherAssert.assertThat;
|
||||
import static org.hamcrest.Matchers.is;
|
||||
|
||||
public class LatchScriptEngine implements ScriptEngineService {
|
||||
|
||||
private static final String NAME = "latch";
|
||||
|
@ -77,7 +81,9 @@ public class LatchScriptEngine implements ScriptEngineService {
|
|||
|
||||
public void finishScriptExecution() throws InterruptedException {
|
||||
scriptCompletionLatch.countDown();
|
||||
scriptCompletionLatch.await(1, TimeUnit.SECONDS);
|
||||
boolean countedDown = scriptCompletionLatch.await(10, TimeUnit.SECONDS);
|
||||
String msg = String.format(Locale.ROOT, "Script completion latch value is [%s], but should be 0", scriptCompletionLatch.getCount());
|
||||
assertThat(msg, countedDown, is(true));
|
||||
}
|
||||
|
||||
public void reset() {
|
||||
|
|
|
@ -20,7 +20,9 @@ import java.security.cert.CertificateException;
|
|||
import java.time.Clock;
|
||||
|
||||
public class TimeWarpedXPackPlugin extends XPackPlugin {
|
||||
private final ClockMock clock = new ClockMock();
|
||||
|
||||
// use a single clock across all nodes using this plugin, this lets keep it static
|
||||
private static final ClockMock clock = new ClockMock();
|
||||
|
||||
public TimeWarpedXPackPlugin(Settings settings) throws IOException,
|
||||
DestroyFailedException, OperatorCreationException, GeneralSecurityException {
|
||||
|
|
|
@ -28,6 +28,7 @@ import java.util.Map;
|
|||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
|
||||
import static org.elasticsearch.xpack.watcher.actions.ActionBuilders.emailAction;
|
||||
import static org.elasticsearch.xpack.watcher.client.WatchSourceBuilders.watchBuilder;
|
||||
import static org.elasticsearch.xpack.watcher.input.InputBuilders.simpleInput;
|
||||
|
@ -107,7 +108,8 @@ public class EmailSecretsIntegrationTests extends AbstractWatcherIntegrationTest
|
|||
assertThat(value, nullValue());
|
||||
|
||||
// now we restart, to make sure the watches and their secrets are reloaded from the index properly
|
||||
assertThat(watcherClient.prepareWatchService().restart().get().isAcknowledged(), is(true));
|
||||
assertAcked(watcherClient.prepareWatchService().stop().get());
|
||||
assertAcked(watcherClient.prepareWatchService().start().get());
|
||||
ensureWatcherStarted();
|
||||
|
||||
// now lets execute the watch manually
|
||||
|
|
|
@ -17,22 +17,28 @@ import org.elasticsearch.xpack.watcher.client.WatcherClient;
|
|||
import org.elasticsearch.xpack.watcher.condition.AlwaysCondition;
|
||||
import org.elasticsearch.xpack.watcher.transport.actions.get.GetWatchResponse;
|
||||
import org.elasticsearch.xpack.watcher.transport.actions.put.PutWatchResponse;
|
||||
import org.elasticsearch.xpack.watcher.transport.actions.stats.WatcherStatsResponse;
|
||||
import org.elasticsearch.xpack.watcher.trigger.schedule.IntervalSchedule;
|
||||
import org.elasticsearch.xpack.watcher.trigger.schedule.IntervalSchedule.Interval;
|
||||
import org.elasticsearch.xpack.watcher.trigger.schedule.ScheduleTrigger;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static org.elasticsearch.common.unit.TimeValue.timeValueSeconds;
|
||||
import static org.hamcrest.Matchers.everyItem;
|
||||
import static org.hamcrest.Matchers.greaterThanOrEqualTo;
|
||||
import static org.hamcrest.Matchers.hasEntry;
|
||||
import static org.hamcrest.Matchers.hasKey;
|
||||
import static org.hamcrest.Matchers.is;
|
||||
import static org.hamcrest.Matchers.not;
|
||||
|
||||
/**
|
||||
* Tests for watcher indexes created before 5.0.
|
||||
*/
|
||||
public class OldWatcherIndicesBackwardsCompatibilityTests extends AbstractOldXPackIndicesBackwardsCompatibilityTestCase {
|
||||
|
||||
@Override
|
||||
public Settings nodeSettings(int ord) {
|
||||
return Settings.builder()
|
||||
|
@ -44,8 +50,12 @@ public class OldWatcherIndicesBackwardsCompatibilityTests extends AbstractOldXPa
|
|||
@Override
|
||||
protected void checkVersion(Version version) throws Exception {
|
||||
// Wait for watcher to actually start....
|
||||
WatcherClient watcherClient = new WatcherClient(client());
|
||||
watcherClient.prepareWatchService().start();
|
||||
assertBusy(() -> {
|
||||
assertEquals(WatcherState.STARTED, internalCluster().getInstance(WatcherService.class).state());
|
||||
List<WatcherState> states = watcherClient.prepareWatcherStats().get().getNodes()
|
||||
.stream().map(WatcherStatsResponse.Node::getWatcherState).collect(Collectors.toList());
|
||||
assertThat(states, everyItem(is(WatcherState.STARTED)));
|
||||
});
|
||||
try {
|
||||
assertWatchIndexContentsWork(version);
|
||||
|
@ -53,10 +63,9 @@ public class OldWatcherIndicesBackwardsCompatibilityTests extends AbstractOldXPa
|
|||
} finally {
|
||||
/* Shut down watcher after every test because watcher can be a bit finicky about shutting down when the node shuts down. This
|
||||
* makes super sure it shuts down *and* causes the test to fail in a sensible spot if it doesn't shut down. */
|
||||
internalCluster().getInstance(WatcherLifeCycleService.class).stop();
|
||||
assertBusy(() -> {
|
||||
assertEquals(WatcherState.STOPPED, internalCluster().getInstance(WatcherService.class).state());
|
||||
});
|
||||
watcherClient.prepareWatchService().stop().get();
|
||||
assertBusy(() -> internalCluster().getInstances(WatcherService.class)
|
||||
.forEach(watcherService -> assertThat(watcherService.state(), is(WatcherState.STOPPED))));
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -0,0 +1,704 @@
|
|||
/*
|
||||
* 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.watcher;
|
||||
|
||||
import org.elasticsearch.ElasticsearchParseException;
|
||||
import org.elasticsearch.Version;
|
||||
import org.elasticsearch.cluster.ClusterChangedEvent;
|
||||
import org.elasticsearch.cluster.ClusterName;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
import org.elasticsearch.cluster.metadata.AliasMetaData;
|
||||
import org.elasticsearch.cluster.metadata.AliasOrIndex;
|
||||
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
||||
import org.elasticsearch.cluster.metadata.MetaData;
|
||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||
import org.elasticsearch.cluster.node.DiscoveryNodes;
|
||||
import org.elasticsearch.cluster.routing.IndexRoutingTable;
|
||||
import org.elasticsearch.cluster.routing.RoutingNode;
|
||||
import org.elasticsearch.cluster.routing.RoutingNodes;
|
||||
import org.elasticsearch.cluster.routing.RoutingTable;
|
||||
import org.elasticsearch.cluster.routing.ShardRouting;
|
||||
import org.elasticsearch.cluster.routing.ShardRoutingState;
|
||||
import org.elasticsearch.cluster.routing.TestShardRouting;
|
||||
import org.elasticsearch.common.bytes.BytesArray;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.engine.Engine;
|
||||
import org.elasticsearch.index.shard.ShardId;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
import org.elasticsearch.xpack.support.clock.ClockMock;
|
||||
import org.elasticsearch.xpack.watcher.WatcherIndexingListener.Configuration;
|
||||
import org.elasticsearch.xpack.watcher.WatcherIndexingListener.ShardAllocationConfiguration;
|
||||
import org.elasticsearch.xpack.watcher.trigger.TriggerService;
|
||||
import org.elasticsearch.xpack.watcher.watch.Watch;
|
||||
import org.elasticsearch.xpack.watcher.watch.WatchStatus;
|
||||
import org.joda.time.DateTime;
|
||||
import org.junit.Before;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.BitSet;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Locale;
|
||||
import java.util.Map;
|
||||
import java.util.SortedMap;
|
||||
import java.util.TreeMap;
|
||||
|
||||
import static java.util.Arrays.asList;
|
||||
import static org.elasticsearch.cluster.routing.ShardRoutingState.RELOCATING;
|
||||
import static org.elasticsearch.cluster.routing.ShardRoutingState.STARTED;
|
||||
import static org.elasticsearch.xpack.watcher.WatcherIndexingListener.INACTIVE;
|
||||
import static org.hamcrest.Matchers.containsString;
|
||||
import static org.hamcrest.Matchers.equalTo;
|
||||
import static org.hamcrest.Matchers.not;
|
||||
import static org.hamcrest.core.Is.is;
|
||||
import static org.joda.time.DateTimeZone.UTC;
|
||||
import static org.mockito.Matchers.any;
|
||||
import static org.mockito.Matchers.anyObject;
|
||||
import static org.mockito.Matchers.eq;
|
||||
import static org.mockito.Mockito.mock;
|
||||
import static org.mockito.Mockito.times;
|
||||
import static org.mockito.Mockito.verify;
|
||||
import static org.mockito.Mockito.verifyZeroInteractions;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
public class WatcherIndexingListenerTests extends ESTestCase {
|
||||
|
||||
private WatcherIndexingListener listener;
|
||||
private Watch.Parser parser = mock(Watch.Parser.class);
|
||||
private ClockMock clock = new ClockMock();
|
||||
private TriggerService triggerService = mock(TriggerService.class);
|
||||
|
||||
private ShardId shardId = mock(ShardId.class);
|
||||
private Engine.IndexResult result = mock(Engine.IndexResult.class);
|
||||
private Engine.Index operation = mock(Engine.Index.class);
|
||||
private Engine.Delete delete = mock(Engine.Delete.class);
|
||||
|
||||
@Before
|
||||
public void setup() throws Exception {
|
||||
clock.freeze();
|
||||
listener = new WatcherIndexingListener(Settings.EMPTY, parser, clock, triggerService);
|
||||
|
||||
Map<ShardId, ShardAllocationConfiguration> map = new HashMap<>();
|
||||
map.put(shardId, new ShardAllocationConfiguration(0, 1, Collections.singletonList("foo")));
|
||||
|
||||
listener.setConfiguration(new Configuration(Watch.INDEX, map));
|
||||
}
|
||||
|
||||
//
|
||||
// tests for document level operations
|
||||
//
|
||||
public void testPreIndexCheckType() throws Exception {
|
||||
when(shardId.getIndexName()).thenReturn(Watch.INDEX);
|
||||
when(operation.type()).thenReturn(randomAlphaOfLength(10));
|
||||
|
||||
Engine.Index index = listener.preIndex(shardId, operation);
|
||||
assertThat(index, is(operation));
|
||||
verifyZeroInteractions(parser);
|
||||
}
|
||||
|
||||
public void testPreIndexCheckIndex() throws Exception {
|
||||
when(operation.type()).thenReturn(Watch.DOC_TYPE);
|
||||
when(shardId.getIndexName()).thenReturn(randomAlphaOfLength(10));
|
||||
|
||||
Engine.Index index = listener.preIndex(shardId, operation);
|
||||
assertThat(index, is(operation));
|
||||
verifyZeroInteractions(parser);
|
||||
}
|
||||
|
||||
public void testPreIndexCheckActive() throws Exception {
|
||||
listener.setConfiguration(INACTIVE);
|
||||
when(operation.type()).thenReturn(Watch.DOC_TYPE);
|
||||
when(shardId.getIndexName()).thenReturn(Watch.INDEX);
|
||||
|
||||
Engine.Index index = listener.preIndex(shardId, operation);
|
||||
assertThat(index, is(operation));
|
||||
verifyZeroInteractions(parser);
|
||||
}
|
||||
|
||||
public void testPreIndex() throws Exception {
|
||||
when(operation.type()).thenReturn(Watch.DOC_TYPE);
|
||||
when(operation.id()).thenReturn(randomAlphaOfLength(10));
|
||||
when(operation.source()).thenReturn(BytesArray.EMPTY);
|
||||
when(shardId.getIndexName()).thenReturn(Watch.INDEX);
|
||||
|
||||
boolean watchActive = randomBoolean();
|
||||
boolean isNewWatch = randomBoolean();
|
||||
Watch watch = mockWatch("_id", watchActive, isNewWatch);
|
||||
when(parser.parseWithSecrets(anyObject(), eq(true), anyObject(), anyObject(), anyObject())).thenReturn(watch);
|
||||
|
||||
Engine.Index returnedOperation = listener.preIndex(shardId, operation);
|
||||
assertThat(returnedOperation, is(operation));
|
||||
|
||||
DateTime now = new DateTime(clock.millis(), UTC);
|
||||
verify(parser).parseWithSecrets(eq(operation.id()), eq(true), eq(BytesArray.EMPTY), eq(now), anyObject());
|
||||
|
||||
if (isNewWatch) {
|
||||
if (watchActive) {
|
||||
verify(triggerService).add(eq(watch));
|
||||
} else {
|
||||
verify(triggerService).remove(eq("_id"));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// this test emulates an index with 10 shards, and ensures that triggering only happens on a
|
||||
// single shard
|
||||
public void testPreIndexWatchGetsOnlyTriggeredOnceAcrossAllShards() throws Exception {
|
||||
String id = randomAlphaOfLength(10);
|
||||
int totalShardCount = randomIntBetween(1, 10);
|
||||
boolean watchActive = randomBoolean();
|
||||
boolean isNewWatch = randomBoolean();
|
||||
Watch watch = mockWatch(id, watchActive, isNewWatch);
|
||||
|
||||
when(shardId.getIndexName()).thenReturn(Watch.INDEX);
|
||||
when(operation.type()).thenReturn(Watch.DOC_TYPE);
|
||||
when(parser.parseWithSecrets(anyObject(), eq(true), anyObject(), anyObject(), anyObject())).thenReturn(watch);
|
||||
|
||||
for (int idx = 0; idx < totalShardCount; idx++) {
|
||||
final Map<ShardId, ShardAllocationConfiguration> localShards = new HashMap<>();
|
||||
localShards.put(shardId, new ShardAllocationConfiguration(idx, totalShardCount, Collections.emptyList()));
|
||||
Configuration configuration = new Configuration(Watch.INDEX, localShards);
|
||||
listener.setConfiguration(configuration);
|
||||
listener.preIndex(shardId, operation);
|
||||
}
|
||||
|
||||
// no matter how many shards we had, this should have been only called once
|
||||
if (isNewWatch) {
|
||||
if (watchActive) {
|
||||
verify(triggerService, times(1)).add(eq(watch));
|
||||
} else {
|
||||
verify(triggerService, times(1)).remove(eq(watch.id()));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private Watch mockWatch(String id, boolean active, boolean isNewWatch) {
|
||||
WatchStatus.State watchState = mock(WatchStatus.State.class);
|
||||
when(watchState.isActive()).thenReturn(active);
|
||||
|
||||
WatchStatus watchStatus = mock(WatchStatus.class);
|
||||
when(watchStatus.state()).thenReturn(watchState);
|
||||
if (isNewWatch) {
|
||||
when(watchStatus.version()).thenReturn(-1L);
|
||||
} else {
|
||||
when(watchStatus.version()).thenReturn(randomLong());
|
||||
}
|
||||
|
||||
Watch watch = mock(Watch.class);
|
||||
when(watch.id()).thenReturn(id);
|
||||
when(watch.status()).thenReturn(watchStatus);
|
||||
|
||||
return watch;
|
||||
}
|
||||
|
||||
public void testPreIndexCheckParsingException() throws Exception {
|
||||
when(operation.type()).thenReturn(Watch.DOC_TYPE);
|
||||
String id = randomAlphaOfLength(10);
|
||||
when(operation.id()).thenReturn(id);
|
||||
when(operation.source()).thenReturn(BytesArray.EMPTY);
|
||||
when(shardId.getIndexName()).thenReturn(Watch.INDEX);
|
||||
when(parser.parseWithSecrets(anyObject(), eq(true), anyObject(), anyObject(), anyObject()))
|
||||
.thenThrow(new IOException("self thrown"));
|
||||
|
||||
ElasticsearchParseException exc = expectThrows(ElasticsearchParseException.class,
|
||||
() -> listener.preIndex(shardId, operation));
|
||||
assertThat(exc.getMessage(), containsString("Could not parse watch"));
|
||||
assertThat(exc.getMessage(), containsString(id));
|
||||
}
|
||||
|
||||
public void testPostIndexRemoveTriggerOnException() throws Exception {
|
||||
when(operation.id()).thenReturn("_id");
|
||||
when(operation.type()).thenReturn(Watch.DOC_TYPE);
|
||||
when(shardId.getIndexName()).thenReturn(Watch.INDEX);
|
||||
|
||||
listener.postIndex(shardId, operation, new ElasticsearchParseException("whatever"));
|
||||
verify(triggerService).remove(eq("_id"));
|
||||
}
|
||||
|
||||
public void testPostIndexDontInvokeForOtherDocuments() throws Exception {
|
||||
when(operation.id()).thenReturn("_id");
|
||||
when(operation.type()).thenReturn(Watch.DOC_TYPE);
|
||||
when(shardId.getIndexName()).thenReturn("anything");
|
||||
when(result.hasFailure()).thenReturn(false);
|
||||
|
||||
listener.postIndex(shardId, operation, new ElasticsearchParseException("whatever"));
|
||||
verifyZeroInteractions(triggerService);
|
||||
}
|
||||
|
||||
public void testPreDeleteCheckActive() throws Exception {
|
||||
listener.setConfiguration(INACTIVE);
|
||||
listener.preDelete(shardId, delete);
|
||||
|
||||
verifyZeroInteractions(triggerService);
|
||||
}
|
||||
|
||||
public void testPreDeleteCheckIndex() throws Exception {
|
||||
when(shardId.getIndexName()).thenReturn(randomAlphaOfLength(10));
|
||||
|
||||
listener.preDelete(shardId, delete);
|
||||
|
||||
verifyZeroInteractions(triggerService);
|
||||
}
|
||||
|
||||
public void testPreDeleteCheckType() throws Exception {
|
||||
when(shardId.getIndexName()).thenReturn(Watch.INDEX);
|
||||
when(delete.type()).thenReturn(randomAlphaOfLength(10));
|
||||
|
||||
listener.preDelete(shardId, delete);
|
||||
|
||||
verifyZeroInteractions(triggerService);
|
||||
}
|
||||
|
||||
public void testPreDelete() throws Exception {
|
||||
when(shardId.getIndexName()).thenReturn(Watch.INDEX);
|
||||
when(delete.type()).thenReturn(Watch.DOC_TYPE);
|
||||
when(delete.id()).thenReturn("_id");
|
||||
|
||||
listener.preDelete(shardId, delete);
|
||||
|
||||
verify(triggerService).remove(eq("_id"));
|
||||
}
|
||||
|
||||
//
|
||||
// tests for cluster state updates
|
||||
//
|
||||
public void testClusterChangedNoMetadata() throws Exception {
|
||||
ClusterState state = mockClusterState(randomAlphaOfLength(10));
|
||||
listener.clusterChanged(new ClusterChangedEvent("any", state, state));
|
||||
|
||||
assertThat(listener.getConfiguration().isIndexAndActive(Watch.INDEX), is(true));
|
||||
}
|
||||
|
||||
public void testClusterChangedNoWatchIndex() throws Exception {
|
||||
Map<ShardId, ShardAllocationConfiguration> map = new HashMap<>();
|
||||
map.put(shardId, new ShardAllocationConfiguration(0, 1, Collections.singletonList("foo")));
|
||||
Configuration randomConfiguration = new Configuration(randomAlphaOfLength(10), map);
|
||||
listener.setConfiguration(randomConfiguration);
|
||||
|
||||
ClusterState clusterState = mockClusterState(null);
|
||||
ClusterChangedEvent clusterChangedEvent = mock(ClusterChangedEvent.class);
|
||||
when(clusterChangedEvent.metaDataChanged()).thenReturn(true);
|
||||
when(clusterChangedEvent.state()).thenReturn(clusterState);
|
||||
|
||||
listener.clusterChanged(clusterChangedEvent);
|
||||
|
||||
assertThat(listener.getConfiguration(), equalTo(INACTIVE));
|
||||
}
|
||||
|
||||
public void testClusterChangedWatchAliasChanged() throws Exception {
|
||||
String newActiveWatchIndex = randomAlphaOfLength(10);
|
||||
RoutingTable routingTable = mock(RoutingTable.class);
|
||||
when(routingTable.hasIndex(eq(newActiveWatchIndex))).thenReturn(true);
|
||||
|
||||
ClusterState currentClusterState = mockClusterState(newActiveWatchIndex);
|
||||
when(currentClusterState.routingTable()).thenReturn(routingTable);
|
||||
DiscoveryNodes nodes = DiscoveryNodes.builder().add(newNode("node_1"))
|
||||
.localNodeId("node_1").build();
|
||||
when(currentClusterState.getNodes()).thenReturn(nodes);
|
||||
RoutingNodes routingNodes = mock(RoutingNodes.class);
|
||||
RoutingNode routingNode = mock(RoutingNode.class);
|
||||
boolean emptyShards = randomBoolean();
|
||||
|
||||
if (emptyShards) {
|
||||
when(routingNode.shardsWithState(eq(newActiveWatchIndex), any()))
|
||||
.thenReturn(Collections.emptyList());
|
||||
} else {
|
||||
Index index = new Index(newActiveWatchIndex, "uuid");
|
||||
ShardId shardId = new ShardId(index, 0);
|
||||
ShardRouting shardRouting = TestShardRouting.newShardRouting(shardId, "node_1", true,
|
||||
STARTED);
|
||||
List<ShardRouting> routing = Collections.singletonList(shardRouting);
|
||||
when(routingNode.shardsWithState(eq(newActiveWatchIndex), eq(STARTED), eq(RELOCATING)))
|
||||
.thenReturn(routing);
|
||||
when(routingTable.allShards(eq(newActiveWatchIndex))).thenReturn(routing);
|
||||
IndexRoutingTable indexRoutingTable = IndexRoutingTable.builder(index)
|
||||
.addShard(shardRouting).build();
|
||||
when(routingTable.index(newActiveWatchIndex)).thenReturn(indexRoutingTable);
|
||||
}
|
||||
|
||||
when(routingNodes.node(eq("node_1"))).thenReturn(routingNode);
|
||||
when(currentClusterState.getRoutingNodes()).thenReturn(routingNodes);
|
||||
|
||||
ClusterState previousClusterState = mockClusterState(randomAlphaOfLength(8));
|
||||
when(previousClusterState.routingTable()).thenReturn(routingTable);
|
||||
|
||||
ClusterChangedEvent event = new ClusterChangedEvent("something", currentClusterState,
|
||||
previousClusterState);
|
||||
listener.clusterChanged(event);
|
||||
|
||||
if (emptyShards) {
|
||||
assertThat(listener.getConfiguration(), is(INACTIVE));
|
||||
} else {
|
||||
assertThat(listener.getConfiguration().isIndexAndActive(newActiveWatchIndex),
|
||||
is(true));
|
||||
}
|
||||
}
|
||||
|
||||
public void testClusterChangedNoRoutingChanges() throws Exception {
|
||||
Index index = new Index(Watch.INDEX, "foo");
|
||||
IndexRoutingTable watchRoutingTable = IndexRoutingTable.builder(index).build();
|
||||
ClusterState previousState = ClusterState.builder(new ClusterName("my-cluster"))
|
||||
.nodes(new DiscoveryNodes.Builder().masterNodeId("node_1").localNodeId("node_1")
|
||||
.add(newNode("node_1")))
|
||||
.routingTable(RoutingTable.builder().add(watchRoutingTable).build())
|
||||
.build();
|
||||
|
||||
ClusterState currentState = ClusterState.builder(new ClusterName("my-cluster"))
|
||||
.nodes(new DiscoveryNodes.Builder().masterNodeId("node_1").localNodeId("node_1")
|
||||
.add(newNode("node_1")).add(newNode("node_2")))
|
||||
.routingTable(RoutingTable.builder().add(watchRoutingTable).build())
|
||||
.build();
|
||||
|
||||
Configuration configuration = listener.getConfiguration();
|
||||
assertThat(configuration.isIndexAndActive(Watch.INDEX), is(true));
|
||||
|
||||
ClusterChangedEvent event = new ClusterChangedEvent("something", currentState,
|
||||
previousState);
|
||||
listener.clusterChanged(event);
|
||||
|
||||
assertThat(listener.getConfiguration(), is(configuration));
|
||||
assertThat(listener.getConfiguration().isIndexAndActive(Watch.INDEX), is(true));
|
||||
}
|
||||
|
||||
// a shard is marked as relocating, no change in the routing yet (replica might be added,
|
||||
// shard might be offloaded)
|
||||
public void testCheckAllocationIdsOnShardStarted() throws Exception {
|
||||
Index index = new Index(Watch.INDEX, "foo");
|
||||
ShardId shardId = new ShardId(index, 0);
|
||||
ShardRoutingState randomState = randomFrom(STARTED, RELOCATING);
|
||||
ShardRouting shardRouting = TestShardRouting.newShardRouting(shardId, "current", true,
|
||||
randomState);
|
||||
IndexRoutingTable indexRoutingTable = IndexRoutingTable.builder(index)
|
||||
.addShard(shardRouting).build();
|
||||
|
||||
Map<ShardId, ShardAllocationConfiguration> allocationIds =
|
||||
listener.getLocalShardAllocationIds(asList(shardRouting), indexRoutingTable);
|
||||
|
||||
assertThat(allocationIds.size(), is(1));
|
||||
assertThat(allocationIds.get(shardId).index, is(0));
|
||||
assertThat(allocationIds.get(shardId).shardCount, is(1));
|
||||
}
|
||||
|
||||
public void testCheckAllocationIdsWithoutShards() throws Exception {
|
||||
Index index = new Index(Watch.INDEX, "foo");
|
||||
ShardId shardId = new ShardId(index, 0);
|
||||
ShardRouting shardRouting = TestShardRouting.newShardRouting(shardId, "other", true,
|
||||
STARTED);
|
||||
IndexRoutingTable indexRoutingTable = IndexRoutingTable.builder(index)
|
||||
.addShard(shardRouting).build();
|
||||
|
||||
Map<ShardId, ShardAllocationConfiguration> allocationIds =
|
||||
listener.getLocalShardAllocationIds(Collections.emptyList(), indexRoutingTable);
|
||||
assertThat(allocationIds.size(), is(0));
|
||||
}
|
||||
|
||||
public void testCheckAllocationIdsWithSeveralShards() {
|
||||
// setup 5 shards, one replica, 10 shards total, all started
|
||||
Index index = new Index(Watch.INDEX, "foo");
|
||||
ShardId firstShardId = new ShardId(index, 0);
|
||||
ShardId secondShardId = new ShardId(index, 1);
|
||||
|
||||
List<ShardRouting> localShards = new ArrayList<>();
|
||||
localShards.add(TestShardRouting.newShardRouting(firstShardId, "node1", true, STARTED));
|
||||
localShards.add(TestShardRouting.newShardRouting(secondShardId, "node1", true, STARTED));
|
||||
|
||||
IndexRoutingTable indexRoutingTable = IndexRoutingTable.builder(index)
|
||||
.addShard(localShards.get(0))
|
||||
.addShard(localShards.get(1))
|
||||
.addShard(TestShardRouting.newShardRouting(firstShardId, "node2", true, STARTED))
|
||||
.addShard(TestShardRouting.newShardRouting(secondShardId, "node2", true, STARTED))
|
||||
.build();
|
||||
|
||||
Map<ShardId, ShardAllocationConfiguration> allocationIds =
|
||||
listener.getLocalShardAllocationIds(localShards, indexRoutingTable);
|
||||
assertThat(allocationIds.size(), is(2));
|
||||
}
|
||||
|
||||
// no matter how many copies of a shard exist, a watch should always be triggered exactly once
|
||||
public void testShardConfigurationShouldBeTriggeredExactlyOnce() throws Exception {
|
||||
// random number of shards
|
||||
int numberOfShards = randomIntBetween(1, 20);
|
||||
int numberOfDocuments = randomIntBetween(1, 10000);
|
||||
BitSet bitSet = new BitSet(numberOfDocuments);
|
||||
logger.info("Testing [{}] documents with [{}] shards", numberOfDocuments, numberOfShards);
|
||||
|
||||
for (int currentShardId = 0; currentShardId < numberOfShards; currentShardId++) {
|
||||
ShardAllocationConfiguration sac = new ShardAllocationConfiguration(currentShardId,
|
||||
numberOfShards, Collections.emptyList());
|
||||
|
||||
for (int i = 0; i < numberOfDocuments; i++) {
|
||||
boolean shouldBeTriggered = sac.shouldBeTriggered("watch_" + i);
|
||||
boolean hasAlreadyBeenTriggered = bitSet.get(i);
|
||||
if (shouldBeTriggered) {
|
||||
String message = String.format(Locale.ROOT, "Watch [%s] has already been " +
|
||||
"triggered", i);
|
||||
assertThat(message, hasAlreadyBeenTriggered, is(false));
|
||||
bitSet.set(i);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
assertThat(bitSet.cardinality(), is(numberOfDocuments));
|
||||
}
|
||||
|
||||
// ensure that non data nodes, deal properly with this cluster state listener
|
||||
public void testOnNonDataNodes() {
|
||||
listener.setConfiguration(INACTIVE);
|
||||
Index index = new Index(Watch.INDEX, "foo");
|
||||
ShardId shardId = new ShardId(index, 0);
|
||||
ShardRouting shardRouting = TestShardRouting.newShardRouting(shardId, "node2", true, STARTED);
|
||||
IndexRoutingTable.Builder indexRoutingTable = IndexRoutingTable.builder(index).addShard(shardRouting);
|
||||
|
||||
DiscoveryNode node1 = new DiscoveryNode("node_1", ESTestCase.buildNewFakeTransportAddress(),
|
||||
Collections.emptyMap(), new HashSet<>(Collections.singletonList(
|
||||
randomFrom(DiscoveryNode.Role.INGEST, DiscoveryNode.Role.MASTER))),
|
||||
Version.CURRENT);
|
||||
|
||||
DiscoveryNode node2 = new DiscoveryNode("node_2", ESTestCase.buildNewFakeTransportAddress(), Collections.emptyMap(),
|
||||
new HashSet<>(Collections.singletonList(DiscoveryNode.Role.DATA)), Version.CURRENT);
|
||||
|
||||
DiscoveryNode node3 = new DiscoveryNode("node_3", ESTestCase.buildNewFakeTransportAddress(), Collections.emptyMap(),
|
||||
new HashSet<>(Collections.singletonList(DiscoveryNode.Role.DATA)), Version.CURRENT);
|
||||
|
||||
IndexMetaData.Builder indexMetaDataBuilder = createIndexBuilder(Watch.INDEX, 1 ,0);
|
||||
|
||||
ClusterState previousState = ClusterState.builder(new ClusterName("my-cluster"))
|
||||
.metaData(MetaData.builder().put(indexMetaDataBuilder))
|
||||
.nodes(new DiscoveryNodes.Builder().masterNodeId("node_1").localNodeId("node_1").add(node1).add(node2).add(node3))
|
||||
.routingTable(RoutingTable.builder().add(indexRoutingTable).build())
|
||||
.build();
|
||||
|
||||
IndexMetaData.Builder newIndexMetaDataBuilder = createIndexBuilder(Watch.INDEX, 1, 1);
|
||||
|
||||
ShardRouting replicaShardRouting = TestShardRouting.newShardRouting(shardId, "node3", false, STARTED);
|
||||
IndexRoutingTable.Builder newRoutingTable = IndexRoutingTable.builder(index)
|
||||
.addShard(shardRouting)
|
||||
.addShard(replicaShardRouting);
|
||||
ClusterState currentState = ClusterState.builder(new ClusterName("my-cluster"))
|
||||
.metaData(MetaData.builder().put(newIndexMetaDataBuilder))
|
||||
.nodes(new DiscoveryNodes.Builder().masterNodeId("node_1").localNodeId("node_1").add(node1).add(node2).add(node3))
|
||||
.routingTable(RoutingTable.builder().add(newRoutingTable).build())
|
||||
.build();
|
||||
|
||||
ClusterChangedEvent event = new ClusterChangedEvent("something", currentState, previousState);
|
||||
listener.clusterChanged(event);
|
||||
assertThat(listener.getConfiguration(), is(INACTIVE));
|
||||
}
|
||||
|
||||
public void testListenerWorksIfOtherIndicesChange() throws Exception {
|
||||
DiscoveryNode node1 = newNode("node_1");
|
||||
DiscoveryNode node2 = newNode("node_2");
|
||||
|
||||
Index index = new Index("random-index", "foo");
|
||||
ShardId firstShardId = new ShardId(index, 0);
|
||||
|
||||
IndexMetaData.Builder indexMetaDataBuilder = createIndexBuilder("random-index", 2, 1);
|
||||
|
||||
IndexRoutingTable.Builder indexRoutingTable = IndexRoutingTable.builder(index)
|
||||
.addShard(TestShardRouting.newShardRouting(firstShardId, "node_1", true, STARTED))
|
||||
.addShard(TestShardRouting.newShardRouting(firstShardId, "node_2", false, STARTED));
|
||||
|
||||
ClusterState previousState = ClusterState.builder(new ClusterName("my-cluster"))
|
||||
.metaData(MetaData.builder().put(indexMetaDataBuilder))
|
||||
.nodes(new DiscoveryNodes.Builder().masterNodeId("node_1").localNodeId("node_1").add(node1).add(node2))
|
||||
.routingTable(RoutingTable.builder().add(indexRoutingTable).build())
|
||||
.build();
|
||||
|
||||
IndexMetaData.Builder currentMetaDataBuilder = createIndexBuilder(Watch.INDEX, 2, 1);
|
||||
|
||||
boolean useWatchIndex = randomBoolean();
|
||||
String indexName = useWatchIndex ? Watch.INDEX : "other-index-name";
|
||||
Index otherIndex = new Index(indexName, "foo");
|
||||
ShardId watchShardId = new ShardId(otherIndex, 0);
|
||||
|
||||
IndexRoutingTable.Builder currentRoutingTable = IndexRoutingTable.builder(otherIndex)
|
||||
.addShard(TestShardRouting.newShardRouting(watchShardId, "node_1", true, STARTED))
|
||||
.addShard(TestShardRouting.newShardRouting(watchShardId, "node_2", false, STARTED));
|
||||
|
||||
ClusterState currentState = ClusterState.builder(new ClusterName("my-cluster"))
|
||||
.metaData(MetaData.builder().put(currentMetaDataBuilder))
|
||||
.nodes(new DiscoveryNodes.Builder().masterNodeId("node_1").localNodeId("node_1").add(node1).add(node2))
|
||||
.routingTable(RoutingTable.builder().add(currentRoutingTable).build())
|
||||
.build();
|
||||
|
||||
listener.setConfiguration(INACTIVE);
|
||||
ClusterChangedEvent event = new ClusterChangedEvent("something", currentState, previousState);
|
||||
listener.clusterChanged(event);
|
||||
if (useWatchIndex) {
|
||||
assertThat(listener.getConfiguration(), is(not(INACTIVE)));
|
||||
} else {
|
||||
assertThat(listener.getConfiguration(), is(INACTIVE));
|
||||
}
|
||||
}
|
||||
|
||||
// 4 nodes, each node has one shard, now node 3 fails, which means only one node should
|
||||
// reload, where as two should not
|
||||
// this test emulates on of those two nodes
|
||||
public void testThatShardConfigurationIsNotReloadedNonAffectedShardsChange() {
|
||||
listener.setConfiguration(INACTIVE);
|
||||
|
||||
DiscoveryNode node1 = newNode("node_1");
|
||||
DiscoveryNode node2 = newNode("node_2");
|
||||
DiscoveryNode node3 = newNode("node_3");
|
||||
DiscoveryNode node4 = newNode("node_4");
|
||||
|
||||
String localNode = randomFrom("node_1", "node_2");
|
||||
|
||||
Index index = new Index(Watch.INDEX, "foo");
|
||||
ShardId firstShardId = new ShardId(index, 0);
|
||||
ShardId secondShardId = new ShardId(index, 1);
|
||||
|
||||
IndexMetaData.Builder indexMetaDataBuilder = createIndexBuilder(Watch.INDEX, 2, 1);
|
||||
|
||||
ShardRouting firstShardRoutingPrimary = TestShardRouting.newShardRouting(firstShardId, "node_1", true, STARTED);
|
||||
ShardRouting firstShardRoutingReplica = TestShardRouting.newShardRouting(firstShardId, "node_2", false, STARTED);
|
||||
ShardRouting secondShardRoutingPrimary = TestShardRouting.newShardRouting(secondShardId, "node_3", true, STARTED);
|
||||
ShardRouting secondShardRoutingReplica = TestShardRouting.newShardRouting(secondShardId, "node_4", false, STARTED);
|
||||
IndexRoutingTable.Builder indexRoutingTable = IndexRoutingTable.builder(index)
|
||||
.addShard(firstShardRoutingPrimary)
|
||||
.addShard(firstShardRoutingReplica)
|
||||
.addShard(secondShardRoutingPrimary)
|
||||
.addShard(secondShardRoutingReplica);
|
||||
|
||||
ClusterState previousState = ClusterState.builder(new ClusterName("my-cluster"))
|
||||
.metaData(MetaData.builder().put(indexMetaDataBuilder))
|
||||
.nodes(new DiscoveryNodes.Builder().masterNodeId("node_1").localNodeId(localNode)
|
||||
.add(node1).add(node2).add(node3).add(node4))
|
||||
.routingTable(RoutingTable.builder().add(indexRoutingTable).build())
|
||||
.build();
|
||||
|
||||
ClusterState emptyState = ClusterState.builder(new ClusterName("my-cluster"))
|
||||
.nodes(new DiscoveryNodes.Builder().masterNodeId("node_1").localNodeId(localNode)
|
||||
.add(node1).add(node2).add(node3).add(node4))
|
||||
.build();
|
||||
|
||||
ClusterChangedEvent event = new ClusterChangedEvent("something", previousState, emptyState);
|
||||
listener.clusterChanged(event);
|
||||
Configuration configuration = listener.getConfiguration();
|
||||
assertThat(configuration, is(not(INACTIVE)));
|
||||
|
||||
// now create a cluster state where node 4 is missing
|
||||
IndexMetaData.Builder newIndexMetaDataBuilder = createIndexBuilder(Watch.INDEX, 2, 1);
|
||||
|
||||
IndexRoutingTable.Builder newRoutingTable = IndexRoutingTable.builder(index)
|
||||
.addShard(firstShardRoutingPrimary)
|
||||
.addShard(firstShardRoutingReplica)
|
||||
.addShard(secondShardRoutingPrimary);
|
||||
|
||||
ClusterState currentState = ClusterState.builder(new ClusterName("my-cluster"))
|
||||
.metaData(MetaData.builder().put(newIndexMetaDataBuilder))
|
||||
.nodes(new DiscoveryNodes.Builder().masterNodeId("node_1").localNodeId(localNode)
|
||||
.add(node1).add(node2).add(node3).add(node4))
|
||||
.routingTable(RoutingTable.builder().add(newRoutingTable).build())
|
||||
.build();
|
||||
|
||||
ClusterChangedEvent nodeGoneEvent = new ClusterChangedEvent("something", currentState, previousState);
|
||||
listener.clusterChanged(nodeGoneEvent);
|
||||
|
||||
// ensure no configuration replacement has happened
|
||||
assertThat(listener.getConfiguration(), is(configuration));
|
||||
}
|
||||
|
||||
// if the creates a .watches alias that points to two indices, set watcher to be inactive
|
||||
public void testWithAliasPointingToTwoIndicesSetsWatcherInactive() {
|
||||
listener.setConfiguration(INACTIVE);
|
||||
DiscoveryNode node1 = newNode("node_1");
|
||||
|
||||
// index foo pointing to .watches
|
||||
Index fooIndex = new Index("foo", "someuuid");
|
||||
ShardId fooShardId = new ShardId(fooIndex, 0);
|
||||
ShardRouting fooShardRouting = TestShardRouting.newShardRouting(fooShardId, node1.getId(), true, STARTED);
|
||||
IndexRoutingTable.Builder fooIndexRoutingTable = IndexRoutingTable.builder(fooIndex).addShard(fooShardRouting);
|
||||
|
||||
// regular cluster state with correct single alias pointing to watches index
|
||||
ClusterState previousState = ClusterState.builder(new ClusterName("my-cluster"))
|
||||
.metaData(MetaData.builder().put(createIndexBuilder("foo", 1, 0)
|
||||
.putAlias(AliasMetaData.builder(Watch.INDEX))))
|
||||
.nodes(new DiscoveryNodes.Builder().masterNodeId("node_1").localNodeId("node_1").add(node1))
|
||||
.routingTable(RoutingTable.builder().add(fooIndexRoutingTable).build())
|
||||
.build();
|
||||
|
||||
// index bar pointing to .watches
|
||||
Index barIndex = new Index("bar", "someuuid2");
|
||||
ShardId barShardId = new ShardId(fooIndex, 0);
|
||||
IndexMetaData.Builder barIndexMetaData = createIndexBuilder("bar", 1, 0).putAlias(AliasMetaData.builder(Watch.INDEX));
|
||||
ShardRouting barShardRouting = TestShardRouting.newShardRouting(barShardId, node1.getId(), true, STARTED);
|
||||
IndexRoutingTable.Builder barIndexRoutingTable = IndexRoutingTable.builder(barIndex).addShard(barShardRouting);
|
||||
|
||||
// cluster state with two indices pointing to the .watches index
|
||||
ClusterState currentState = ClusterState.builder(new ClusterName("my-cluster"))
|
||||
.metaData(MetaData.builder().put(createIndexBuilder("foo", 1, 0)
|
||||
.putAlias(AliasMetaData.builder(Watch.INDEX)))
|
||||
.put(barIndexMetaData))
|
||||
.nodes(new DiscoveryNodes.Builder().masterNodeId("node_1").localNodeId("node_1")
|
||||
.add(node1))
|
||||
.routingTable(RoutingTable.builder()
|
||||
.add(IndexRoutingTable.builder(fooIndex).addShard(fooShardRouting))
|
||||
.add(barIndexRoutingTable).build())
|
||||
.build();
|
||||
|
||||
ClusterChangedEvent nodeGoneEvent = new ClusterChangedEvent("something", currentState, previousState);
|
||||
listener.clusterChanged(nodeGoneEvent);
|
||||
|
||||
// ensure no configuration replacement has happened
|
||||
assertThat(listener.getConfiguration(), is(INACTIVE));
|
||||
}
|
||||
|
||||
//
|
||||
// helper methods
|
||||
//
|
||||
/**
|
||||
* create a mock cluster state, the returns the specified index as watch index
|
||||
*/
|
||||
private ClusterState mockClusterState(String watchIndex) {
|
||||
MetaData metaData = mock(MetaData.class);
|
||||
if (watchIndex == null) {
|
||||
when(metaData.getAliasAndIndexLookup()).thenReturn(Collections.emptySortedMap());
|
||||
} else {
|
||||
SortedMap<String, AliasOrIndex> indices = new TreeMap<>();
|
||||
|
||||
IndexMetaData indexMetaData = mock(IndexMetaData.class);
|
||||
when(indexMetaData.getIndex()).thenReturn(new Index(watchIndex, randomAlphaOfLength(10)));
|
||||
indices.put(watchIndex, new AliasOrIndex.Index(indexMetaData));
|
||||
|
||||
// now point the alias, if the watch index is not .watches
|
||||
if (watchIndex.equals(Watch.INDEX) == false) {
|
||||
AliasMetaData aliasMetaData = mock(AliasMetaData.class);
|
||||
when(aliasMetaData.alias()).thenReturn(watchIndex);
|
||||
indices.put(Watch.INDEX, new AliasOrIndex.Alias(aliasMetaData, indexMetaData));
|
||||
}
|
||||
|
||||
when(metaData.getAliasAndIndexLookup()).thenReturn(indices);
|
||||
}
|
||||
|
||||
ClusterState clusterState = mock(ClusterState.class);
|
||||
when(clusterState.metaData()).thenReturn(metaData);
|
||||
|
||||
DiscoveryNodes nodes = DiscoveryNodes.builder().localNodeId("node_1").masterNodeId("node_1").add(newNode("node_1")).build();
|
||||
when(clusterState.nodes()).thenReturn(nodes);
|
||||
|
||||
return clusterState;
|
||||
}
|
||||
|
||||
private IndexMetaData.Builder createIndexBuilder(String name, int numberOfShards,
|
||||
int numberOfReplicas) {
|
||||
return IndexMetaData.builder(name)
|
||||
.settings(Settings.builder()
|
||||
.put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, numberOfShards)
|
||||
.put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, numberOfReplicas)
|
||||
.put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT)
|
||||
);
|
||||
}
|
||||
|
||||
private static DiscoveryNode newNode(String nodeId) {
|
||||
return new DiscoveryNode(nodeId, ESTestCase.buildNewFakeTransportAddress(), Collections.emptyMap(),
|
||||
new HashSet<>(asList(DiscoveryNode.Role.values())), Version.CURRENT);
|
||||
}
|
||||
}
|
|
@ -14,22 +14,34 @@ import org.elasticsearch.cluster.ClusterStateUpdateTask;
|
|||
import org.elasticsearch.cluster.block.ClusterBlocks;
|
||||
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
||||
import org.elasticsearch.cluster.metadata.MetaData;
|
||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||
import org.elasticsearch.cluster.node.DiscoveryNodes;
|
||||
import org.elasticsearch.cluster.routing.IndexRoutingTable;
|
||||
import org.elasticsearch.cluster.routing.RoutingTable;
|
||||
import org.elasticsearch.cluster.routing.ShardRouting;
|
||||
import org.elasticsearch.cluster.routing.TestShardRouting;
|
||||
import org.elasticsearch.cluster.service.ClusterService;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.util.concurrent.EsExecutors;
|
||||
import org.elasticsearch.gateway.GatewayService;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.shard.ShardId;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
import org.elasticsearch.xpack.watcher.watch.Watch;
|
||||
import org.junit.Before;
|
||||
import org.mockito.invocation.InvocationOnMock;
|
||||
import org.mockito.stubbing.Answer;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.HashSet;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
|
||||
import static java.util.Arrays.asList;
|
||||
import static org.elasticsearch.cluster.routing.ShardRoutingState.RELOCATING;
|
||||
import static org.elasticsearch.cluster.routing.ShardRoutingState.STARTED;
|
||||
import static org.mockito.Matchers.any;
|
||||
import static org.mockito.Matchers.anyString;
|
||||
import static org.mockito.Matchers.eq;
|
||||
import static org.mockito.Mockito.doAnswer;
|
||||
import static org.mockito.Mockito.mock;
|
||||
import static org.mockito.Mockito.never;
|
||||
|
@ -38,6 +50,7 @@ import static org.mockito.Mockito.verify;
|
|||
import static org.mockito.Mockito.when;
|
||||
|
||||
public class WatcherLifeCycleServiceTests extends ESTestCase {
|
||||
|
||||
private ClusterService clusterService;
|
||||
private WatcherService watcherService;
|
||||
private WatcherLifeCycleService lifeCycleService;
|
||||
|
@ -48,13 +61,10 @@ public class WatcherLifeCycleServiceTests extends ESTestCase {
|
|||
final ExecutorService executorService = EsExecutors.newDirectExecutorService();
|
||||
when(threadPool.executor(anyString())).thenReturn(executorService);
|
||||
clusterService = mock(ClusterService.class);
|
||||
Answer<Object> answer = new Answer<Object>() {
|
||||
@Override
|
||||
public Object answer(InvocationOnMock invocationOnMock) throws Throwable {
|
||||
AckedClusterStateUpdateTask updateTask = (AckedClusterStateUpdateTask) invocationOnMock.getArguments()[1];
|
||||
updateTask.onAllNodesAcked(null);
|
||||
return null;
|
||||
}
|
||||
Answer<Object> answer = invocationOnMock -> {
|
||||
AckedClusterStateUpdateTask updateTask = (AckedClusterStateUpdateTask) invocationOnMock.getArguments()[1];
|
||||
updateTask.onAllNodesAcked(null);
|
||||
return null;
|
||||
};
|
||||
doAnswer(answer).when(clusterService).submitStateUpdateTask(anyString(), any(ClusterStateUpdateTask.class));
|
||||
watcherService = mock(WatcherService.class);
|
||||
|
@ -62,29 +72,29 @@ public class WatcherLifeCycleServiceTests extends ESTestCase {
|
|||
}
|
||||
|
||||
public void testStartAndStopCausedByClusterState() throws Exception {
|
||||
// starting... local node is master node
|
||||
DiscoveryNodes.Builder nodes = new DiscoveryNodes.Builder().masterNodeId("id1").localNodeId("id1");
|
||||
IndexRoutingTable indexRoutingTable = IndexRoutingTable.builder(new Index("anything", "foo")).build();
|
||||
ClusterState previousClusterState = ClusterState.builder(new ClusterName("my-cluster"))
|
||||
.nodes(new DiscoveryNodes.Builder().masterNodeId("node_1").localNodeId("node_1").add(newNode("node_1")))
|
||||
.routingTable(RoutingTable.builder().add(indexRoutingTable).build())
|
||||
.build();
|
||||
|
||||
IndexRoutingTable watchRoutingTable = IndexRoutingTable.builder(new Index(Watch.INDEX, "foo")).build();
|
||||
ClusterState clusterState = ClusterState.builder(new ClusterName("my-cluster"))
|
||||
.nodes(nodes).build();
|
||||
.nodes(new DiscoveryNodes.Builder().masterNodeId("node_1").localNodeId("node_1").add(newNode("node_1")))
|
||||
.routingTable(RoutingTable.builder().add(watchRoutingTable).build())
|
||||
.build();
|
||||
|
||||
when(watcherService.state()).thenReturn(WatcherState.STOPPED);
|
||||
when(watcherService.validate(clusterState)).thenReturn(true);
|
||||
lifeCycleService.applyClusterState(new ClusterChangedEvent("any", clusterState, clusterState));
|
||||
lifeCycleService.clusterChanged(new ClusterChangedEvent("any", clusterState, previousClusterState));
|
||||
verify(watcherService, times(1)).start(clusterState);
|
||||
verify(watcherService, never()).stop();
|
||||
|
||||
// Trying to start a second time, but that should have no affect.
|
||||
when(watcherService.state()).thenReturn(WatcherState.STARTED);
|
||||
lifeCycleService.applyClusterState(new ClusterChangedEvent("any", clusterState, clusterState));
|
||||
lifeCycleService.clusterChanged(new ClusterChangedEvent("any", clusterState, previousClusterState));
|
||||
verify(watcherService, times(1)).start(clusterState);
|
||||
verify(watcherService, never()).stop();
|
||||
|
||||
// Stopping because local node is no longer master node
|
||||
nodes = new DiscoveryNodes.Builder().masterNodeId("id1").localNodeId("id2");
|
||||
ClusterState noMasterClusterState = ClusterState.builder(new ClusterName("my-cluster"))
|
||||
.nodes(nodes).build();
|
||||
lifeCycleService.applyClusterState(new ClusterChangedEvent("any", noMasterClusterState, noMasterClusterState));
|
||||
verify(watcherService, times(1)).stop();
|
||||
verify(watcherService, times(1)).start(clusterState);
|
||||
}
|
||||
|
||||
public void testStartWithStateNotRecoveredBlock() throws Exception {
|
||||
|
@ -93,14 +103,17 @@ public class WatcherLifeCycleServiceTests extends ESTestCase {
|
|||
.blocks(ClusterBlocks.builder().addGlobalBlock(GatewayService.STATE_NOT_RECOVERED_BLOCK))
|
||||
.nodes(nodes).build();
|
||||
when(watcherService.state()).thenReturn(WatcherState.STOPPED);
|
||||
lifeCycleService.applyClusterState(new ClusterChangedEvent("any", clusterState, clusterState));
|
||||
lifeCycleService.clusterChanged(new ClusterChangedEvent("any", clusterState, clusterState));
|
||||
verify(watcherService, never()).start(any(ClusterState.class));
|
||||
}
|
||||
|
||||
public void testManualStartStop() throws Exception {
|
||||
DiscoveryNodes.Builder nodes = new DiscoveryNodes.Builder().masterNodeId("id1").localNodeId("id1");
|
||||
IndexRoutingTable watchRoutingTable = IndexRoutingTable.builder(new Index(Watch.INDEX, "foo")).build();
|
||||
ClusterState clusterState = ClusterState.builder(new ClusterName("my-cluster"))
|
||||
.nodes(nodes).build();
|
||||
.nodes(new DiscoveryNodes.Builder().masterNodeId("node_1").localNodeId("node_1").add(newNode("node_1")))
|
||||
.routingTable(RoutingTable.builder().add(watchRoutingTable).build())
|
||||
.build();
|
||||
|
||||
when(clusterService.state()).thenReturn(clusterState);
|
||||
when(watcherService.validate(clusterState)).thenReturn(true);
|
||||
|
||||
|
@ -116,33 +129,27 @@ public class WatcherLifeCycleServiceTests extends ESTestCase {
|
|||
|
||||
// Starting via cluster state update, we shouldn't start because we have been stopped manually.
|
||||
when(watcherService.state()).thenReturn(WatcherState.STOPPED);
|
||||
lifeCycleService.applyClusterState(new ClusterChangedEvent("any", clusterState, clusterState));
|
||||
verify(watcherService, times(1)).start(any(ClusterState.class));
|
||||
verify(watcherService, times(1)).stop();
|
||||
|
||||
// we can only start, if we start manually
|
||||
lifeCycleService.start();
|
||||
lifeCycleService.clusterChanged(new ClusterChangedEvent("any", clusterState, clusterState));
|
||||
verify(watcherService, times(2)).start(any(ClusterState.class));
|
||||
verify(watcherService, times(1)).stop();
|
||||
|
||||
// stop watcher via cluster state update
|
||||
nodes = new DiscoveryNodes.Builder().masterNodeId("id1").localNodeId("id2");
|
||||
clusterState = ClusterState.builder(new ClusterName("my-cluster"))
|
||||
.nodes(nodes).build();
|
||||
// no change, keep going
|
||||
clusterState = ClusterState.builder(new ClusterName("my-cluster"))
|
||||
.nodes(new DiscoveryNodes.Builder().masterNodeId("node_1").localNodeId("node_1").add(newNode("node_1")))
|
||||
.build();
|
||||
when(watcherService.state()).thenReturn(WatcherState.STARTED);
|
||||
lifeCycleService.applyClusterState(new ClusterChangedEvent("any", clusterState, clusterState));
|
||||
lifeCycleService.clusterChanged(new ClusterChangedEvent("any", clusterState, clusterState));
|
||||
verify(watcherService, times(2)).start(any(ClusterState.class));
|
||||
verify(watcherService, times(2)).stop();
|
||||
verify(watcherService, times(1)).stop();
|
||||
|
||||
// starting watcher via cluster state update, which should work, because we manually started before
|
||||
nodes = new DiscoveryNodes.Builder().masterNodeId("id1").localNodeId("id1");
|
||||
clusterState = ClusterState.builder(new ClusterName("my-cluster"))
|
||||
.nodes(nodes).build();
|
||||
ClusterState previousClusterState = ClusterState.builder(new ClusterName("my-cluster"))
|
||||
.nodes(new DiscoveryNodes.Builder().masterNodeId("node_1").localNodeId("node_1").add(newNode("node_1")))
|
||||
.build();
|
||||
when(watcherService.validate(clusterState)).thenReturn(true);
|
||||
when(watcherService.state()).thenReturn(WatcherState.STOPPED);
|
||||
lifeCycleService.applyClusterState(new ClusterChangedEvent("any", clusterState, clusterState));
|
||||
lifeCycleService.clusterChanged(new ClusterChangedEvent("any", clusterState, previousClusterState));
|
||||
verify(watcherService, times(3)).start(any(ClusterState.class));
|
||||
verify(watcherService, times(2)).stop();
|
||||
verify(watcherService, times(1)).stop();
|
||||
}
|
||||
|
||||
public void testManualStartStopClusterStateNotValid() throws Exception {
|
||||
|
@ -153,7 +160,6 @@ public class WatcherLifeCycleServiceTests extends ESTestCase {
|
|||
when(watcherService.state()).thenReturn(WatcherState.STOPPED);
|
||||
when(watcherService.validate(clusterState)).thenReturn(false);
|
||||
|
||||
|
||||
lifeCycleService.start();
|
||||
verify(watcherService, never()).start(any(ClusterState.class));
|
||||
verify(watcherService, never()).stop();
|
||||
|
@ -172,45 +178,134 @@ public class WatcherLifeCycleServiceTests extends ESTestCase {
|
|||
verify(watcherService, never()).stop();
|
||||
}
|
||||
|
||||
public void testWatchIndexDeletion() throws Exception {
|
||||
DiscoveryNodes discoveryNodes = new DiscoveryNodes.Builder().masterNodeId("id1").localNodeId("id1").build();
|
||||
// old cluster state that contains watcher index
|
||||
Settings indexSettings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build();
|
||||
ClusterState oldClusterState = ClusterState.builder(new ClusterName("my-cluster"))
|
||||
.metaData(new MetaData.Builder().put(IndexMetaData.builder(Watch.INDEX)
|
||||
.settings(indexSettings).numberOfReplicas(0).numberOfShards(1)))
|
||||
.nodes(discoveryNodes).build();
|
||||
public void testNoLocalShards() throws Exception {
|
||||
Index watchIndex = new Index(Watch.INDEX, "foo");
|
||||
ShardId shardId = new ShardId(watchIndex, 0);
|
||||
IndexRoutingTable watchRoutingTable = IndexRoutingTable.builder(watchIndex)
|
||||
.addShard(TestShardRouting.newShardRouting(shardId, "node_2", true, randomFrom(STARTED, RELOCATING)))
|
||||
.build();
|
||||
DiscoveryNodes nodes = new DiscoveryNodes.Builder().masterNodeId("node_1").localNodeId("node_1")
|
||||
.add(newNode("node_1")).add(newNode("node_2"))
|
||||
.build();
|
||||
ClusterState clusterState = ClusterState.builder(new ClusterName("my-cluster"))
|
||||
.nodes(nodes)
|
||||
.routingTable(RoutingTable.builder().add(watchRoutingTable).build())
|
||||
.build();
|
||||
|
||||
// new cluster state that does not contain watcher index
|
||||
ClusterState newClusterState = ClusterState.builder(new ClusterName("my-cluster")).nodes(discoveryNodes).build();
|
||||
when(watcherService.state()).thenReturn(WatcherState.STARTED);
|
||||
|
||||
lifeCycleService.applyClusterState(new ClusterChangedEvent("any", newClusterState, oldClusterState));
|
||||
verify(watcherService, never()).start(any(ClusterState.class));
|
||||
verify(watcherService, never()).stop();
|
||||
verify(watcherService, times(1)).watchIndexDeletedOrClosed();
|
||||
lifeCycleService.clusterChanged(new ClusterChangedEvent("any", clusterState, clusterState));
|
||||
verify(watcherService).pauseExecution();
|
||||
}
|
||||
|
||||
public void testWatchIndexClosing() throws Exception {
|
||||
DiscoveryNodes discoveryNodes = new DiscoveryNodes.Builder().masterNodeId("id1").localNodeId("id1").build();
|
||||
// old cluster state that contains watcher index
|
||||
Settings indexSettings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build();
|
||||
ClusterState oldClusterState = ClusterState.builder(new ClusterName("my-cluster"))
|
||||
.metaData(new MetaData.Builder().put(IndexMetaData.builder(Watch.INDEX)
|
||||
.settings(indexSettings).numberOfReplicas(0).numberOfShards(1)))
|
||||
.nodes(discoveryNodes).build();
|
||||
public void testReplicaWasAddedOrRemoved() throws Exception {
|
||||
Index watchIndex = new Index(Watch.INDEX, "foo");
|
||||
ShardId shardId = new ShardId(watchIndex, 0);
|
||||
ShardId secondShardId = new ShardId(watchIndex, 1);
|
||||
DiscoveryNodes discoveryNodes = new DiscoveryNodes.Builder().masterNodeId("node_1").localNodeId("node_1")
|
||||
.add(newNode("node_1"))
|
||||
.add(newNode("node_2"))
|
||||
.build();
|
||||
|
||||
IndexRoutingTable previousWatchRoutingTable = IndexRoutingTable.builder(watchIndex)
|
||||
.addShard(TestShardRouting.newShardRouting(secondShardId, "node_1", true, randomFrom(STARTED, RELOCATING)))
|
||||
.addShard(TestShardRouting.newShardRouting(shardId, "node_2", true, randomFrom(STARTED, RELOCATING)))
|
||||
.build();
|
||||
|
||||
IndexMetaData indexMetaData = IndexMetaData.builder(Watch.INDEX)
|
||||
.settings(Settings.builder()
|
||||
.put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1)
|
||||
.put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 0)
|
||||
.put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT)
|
||||
).build();
|
||||
|
||||
ClusterState stateWithPrimaryShard = ClusterState.builder(new ClusterName("my-cluster"))
|
||||
.nodes(discoveryNodes)
|
||||
.routingTable(RoutingTable.builder().add(previousWatchRoutingTable).build())
|
||||
.metaData(MetaData.builder().put(indexMetaData, false))
|
||||
.build();
|
||||
|
||||
IndexRoutingTable currentWatchRoutingTable = IndexRoutingTable.builder(watchIndex)
|
||||
.addShard(TestShardRouting.newShardRouting(shardId, "node_1", false, randomFrom(STARTED, RELOCATING)))
|
||||
.addShard(TestShardRouting.newShardRouting(secondShardId, "node_1", true, randomFrom(STARTED, RELOCATING)))
|
||||
.addShard(TestShardRouting.newShardRouting(shardId, "node_2", true, STARTED))
|
||||
.build();
|
||||
|
||||
ClusterState stateWithReplicaAdded = ClusterState.builder(new ClusterName("my-cluster"))
|
||||
.nodes(discoveryNodes)
|
||||
.routingTable(RoutingTable.builder().add(currentWatchRoutingTable).build())
|
||||
.metaData(MetaData.builder().put(indexMetaData, false))
|
||||
.build();
|
||||
|
||||
// randomize between addition or removal of a replica
|
||||
boolean replicaAdded = randomBoolean();
|
||||
ClusterChangedEvent event;
|
||||
ClusterState usedClusterState;
|
||||
if (replicaAdded) {
|
||||
event = new ClusterChangedEvent("any", stateWithReplicaAdded, stateWithPrimaryShard);
|
||||
usedClusterState = stateWithReplicaAdded;
|
||||
} else {
|
||||
event = new ClusterChangedEvent("any", stateWithPrimaryShard, stateWithReplicaAdded);
|
||||
usedClusterState = stateWithPrimaryShard;
|
||||
}
|
||||
|
||||
// new cluster state with a closed watcher index
|
||||
ClusterState newClusterState = ClusterState.builder(new ClusterName("my-cluster"))
|
||||
.metaData(new MetaData.Builder().put(IndexMetaData.builder(Watch.INDEX).state(IndexMetaData.State.CLOSE)
|
||||
.settings(indexSettings).numberOfReplicas(0).numberOfShards(1)))
|
||||
.nodes(discoveryNodes).build();
|
||||
when(watcherService.state()).thenReturn(WatcherState.STARTED);
|
||||
|
||||
lifeCycleService.applyClusterState(new ClusterChangedEvent("any", newClusterState, oldClusterState));
|
||||
verify(watcherService, never()).start(any(ClusterState.class));
|
||||
verify(watcherService, never()).stop();
|
||||
verify(watcherService, times(1)).watchIndexDeletedOrClosed();
|
||||
lifeCycleService.clusterChanged(event);
|
||||
verify(watcherService).reload(eq(usedClusterState));
|
||||
}
|
||||
|
||||
// make sure that cluster state changes can be processed on nodes that do not hold data
|
||||
public void testNonDataNode() {
|
||||
Index index = new Index(Watch.INDEX, "foo");
|
||||
ShardId shardId = new ShardId(index, 0);
|
||||
ShardRouting shardRouting = TestShardRouting.newShardRouting(shardId, "node2", true, STARTED);
|
||||
IndexRoutingTable.Builder indexRoutingTable = IndexRoutingTable.builder(index).addShard(shardRouting);
|
||||
|
||||
DiscoveryNode node1 = new DiscoveryNode("node_1", ESTestCase.buildNewFakeTransportAddress(), Collections.emptyMap(),
|
||||
new HashSet<>(asList(randomFrom(DiscoveryNode.Role.INGEST, DiscoveryNode.Role.MASTER))), Version.CURRENT);
|
||||
|
||||
DiscoveryNode node2 = new DiscoveryNode("node_2", ESTestCase.buildNewFakeTransportAddress(), Collections.emptyMap(),
|
||||
new HashSet<>(asList(DiscoveryNode.Role.DATA)), Version.CURRENT);
|
||||
|
||||
DiscoveryNode node3 = new DiscoveryNode("node_3", ESTestCase.buildNewFakeTransportAddress(), Collections.emptyMap(),
|
||||
new HashSet<>(asList(DiscoveryNode.Role.DATA)), Version.CURRENT);
|
||||
|
||||
IndexMetaData.Builder indexMetaDataBuilder = IndexMetaData.builder(Watch.INDEX)
|
||||
.settings(Settings.builder()
|
||||
.put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1)
|
||||
.put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 0)
|
||||
.put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT)
|
||||
);
|
||||
|
||||
ClusterState previousState = ClusterState.builder(new ClusterName("my-cluster"))
|
||||
.metaData(MetaData.builder().put(indexMetaDataBuilder))
|
||||
.nodes(new DiscoveryNodes.Builder().masterNodeId("node_1").localNodeId("node_1").add(node1).add(node2).add(node3))
|
||||
.routingTable(RoutingTable.builder().add(indexRoutingTable).build())
|
||||
.build();
|
||||
|
||||
IndexMetaData.Builder newIndexMetaDataBuilder = IndexMetaData.builder(Watch.INDEX)
|
||||
.settings(Settings.builder()
|
||||
.put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1)
|
||||
.put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 1)
|
||||
.put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT)
|
||||
);
|
||||
|
||||
ShardRouting replicaShardRouting = TestShardRouting.newShardRouting(shardId, "node3", false, STARTED);
|
||||
IndexRoutingTable.Builder newRoutingTable = IndexRoutingTable.builder(index).addShard(shardRouting).addShard(replicaShardRouting);
|
||||
ClusterState currentState = ClusterState.builder(new ClusterName("my-cluster"))
|
||||
.metaData(MetaData.builder().put(newIndexMetaDataBuilder))
|
||||
.nodes(new DiscoveryNodes.Builder().masterNodeId("node_1").localNodeId("node_1").add(node1).add(node2).add(node3))
|
||||
.routingTable(RoutingTable.builder().add(newRoutingTable).build())
|
||||
.build();
|
||||
|
||||
when(watcherService.state()).thenReturn(WatcherState.STARTED);
|
||||
lifeCycleService.clusterChanged(new ClusterChangedEvent("any", currentState, previousState));
|
||||
verify(watcherService, times(0)).pauseExecution();
|
||||
verify(watcherService, times(0)).reload(any());
|
||||
}
|
||||
|
||||
private static DiscoveryNode newNode(String nodeName) {
|
||||
return new DiscoveryNode(nodeName, ESTestCase.buildNewFakeTransportAddress(), Collections.emptyMap(),
|
||||
new HashSet<>(asList(DiscoveryNode.Role.values())), Version.CURRENT);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -13,28 +13,29 @@ import static org.hamcrest.Matchers.containsString;
|
|||
public class WatcherPluginTests extends ESTestCase {
|
||||
|
||||
public void testValidAutoCreateIndex() {
|
||||
Watcher.validAutoCreateIndex(Settings.EMPTY);
|
||||
Watcher.validAutoCreateIndex(Settings.builder().put("action.auto_create_index", true).build());
|
||||
Watcher.validAutoCreateIndex(Settings.EMPTY, logger);
|
||||
Watcher.validAutoCreateIndex(Settings.builder().put("action.auto_create_index", true).build(), logger);
|
||||
|
||||
IllegalArgumentException exception = expectThrows(IllegalArgumentException.class,
|
||||
() -> Watcher.validAutoCreateIndex(Settings.builder().put("action.auto_create_index", false).build()));
|
||||
() -> Watcher.validAutoCreateIndex(Settings.builder().put("action.auto_create_index", false).build(), logger));
|
||||
assertThat(exception.getMessage(), containsString("[.watches, .triggered_watches, .watcher-history-*]"));
|
||||
|
||||
Watcher.validAutoCreateIndex(Settings.builder().put("action.auto_create_index",
|
||||
".watches,.triggered_watches,.watcher-history*").build());
|
||||
Watcher.validAutoCreateIndex(Settings.builder().put("action.auto_create_index", "*w*").build());
|
||||
Watcher.validAutoCreateIndex(Settings.builder().put("action.auto_create_index", ".w*,.t*").build());
|
||||
".watches,.triggered_watches,.watcher-history*").build(), logger);
|
||||
Watcher.validAutoCreateIndex(Settings.builder().put("action.auto_create_index", "*w*").build(), logger);
|
||||
Watcher.validAutoCreateIndex(Settings.builder().put("action.auto_create_index", ".w*,.t*").build(), logger);
|
||||
|
||||
exception = expectThrows(IllegalArgumentException.class,
|
||||
() -> Watcher.validAutoCreateIndex(Settings.builder().put("action.auto_create_index", ".watches").build()));
|
||||
() -> Watcher.validAutoCreateIndex(Settings.builder().put("action.auto_create_index", ".watches").build(), logger));
|
||||
assertThat(exception.getMessage(), containsString("[.watches, .triggered_watches, .watcher-history-*]"));
|
||||
|
||||
exception = expectThrows(IllegalArgumentException.class,
|
||||
() -> Watcher.validAutoCreateIndex(Settings.builder().put("action.auto_create_index", ".triggered_watch").build()));
|
||||
() -> Watcher.validAutoCreateIndex(Settings.builder().put("action.auto_create_index", ".triggered_watch").build(), logger));
|
||||
assertThat(exception.getMessage(), containsString("[.watches, .triggered_watches, .watcher-history-*]"));
|
||||
|
||||
exception = expectThrows(IllegalArgumentException.class,
|
||||
() -> Watcher.validAutoCreateIndex(Settings.builder().put("action.auto_create_index", ".watcher-history-*").build()));
|
||||
() -> Watcher.validAutoCreateIndex(Settings.builder().put("action.auto_create_index", ".watcher-history-*").build(),
|
||||
logger));
|
||||
assertThat(exception.getMessage(), containsString("[.watches, .triggered_watches, .watcher-history-*]"));
|
||||
}
|
||||
|
||||
|
|
|
@ -0,0 +1,51 @@
|
|||
/*
|
||||
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||
* or more contributor license agreements. Licensed under the Elastic License;
|
||||
* you may not use this file except in compliance with the Elastic License.
|
||||
*/
|
||||
package org.elasticsearch.xpack.watcher;
|
||||
|
||||
import org.elasticsearch.Version;
|
||||
import org.elasticsearch.cluster.ClusterName;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
||||
import org.elasticsearch.cluster.metadata.MetaData;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
import org.elasticsearch.xpack.watcher.execution.ExecutionService;
|
||||
import org.elasticsearch.xpack.watcher.execution.TriggeredWatchStore;
|
||||
import org.elasticsearch.xpack.watcher.support.init.proxy.WatcherClientProxy;
|
||||
import org.elasticsearch.xpack.watcher.trigger.TriggerService;
|
||||
import org.elasticsearch.xpack.watcher.watch.Watch;
|
||||
|
||||
import static org.hamcrest.Matchers.is;
|
||||
import static org.mockito.Matchers.anyObject;
|
||||
import static org.mockito.Mockito.mock;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
public class WatcherServiceTests extends ESTestCase {
|
||||
|
||||
public void testValidateStartWithClosedIndex() throws Exception {
|
||||
TriggerService triggerService = mock(TriggerService.class);
|
||||
TriggeredWatchStore triggeredWatchStore = mock(TriggeredWatchStore.class);
|
||||
ExecutionService executionService = mock(ExecutionService.class);
|
||||
when(executionService.validate(anyObject())).thenReturn(true);
|
||||
Watch.Parser parser = mock(Watch.Parser.class);
|
||||
WatcherClientProxy client = mock(WatcherClientProxy.class);
|
||||
|
||||
WatcherService service = new WatcherService(Settings.EMPTY, triggerService, triggeredWatchStore,
|
||||
executionService, parser, client);
|
||||
|
||||
ClusterState.Builder csBuilder = new ClusterState.Builder(new ClusterName("_name"));
|
||||
MetaData.Builder metaDataBuilder = MetaData.builder();
|
||||
Settings indexSettings = settings(Version.CURRENT)
|
||||
.put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1)
|
||||
.put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 1)
|
||||
.build();
|
||||
metaDataBuilder.put(IndexMetaData.builder(Watch.INDEX).state(IndexMetaData.State.CLOSE).settings(indexSettings));
|
||||
csBuilder.metaData(metaDataBuilder);
|
||||
|
||||
assertThat(service.validate(csBuilder.build()), is(false));
|
||||
}
|
||||
|
||||
}
|
|
@ -47,7 +47,7 @@ public class ActionErrorIntegrationTests extends AbstractWatcherIntegrationTestC
|
|||
|
||||
assertThat(putWatchResponse.isCreated(), is(true));
|
||||
|
||||
timeWarp().scheduler().trigger("_id");
|
||||
timeWarp().trigger("_id");
|
||||
|
||||
flush();
|
||||
|
||||
|
@ -64,7 +64,7 @@ public class ActionErrorIntegrationTests extends AbstractWatcherIntegrationTestC
|
|||
|
||||
// within the 60 minute throttling period
|
||||
timeWarp().clock().fastForward(TimeValue.timeValueMinutes(randomIntBetween(1, 50)));
|
||||
timeWarp().scheduler().trigger("_id");
|
||||
timeWarp().trigger("_id");
|
||||
|
||||
flush();
|
||||
|
||||
|
|
|
@ -7,9 +7,12 @@ package org.elasticsearch.xpack.watcher.actions;
|
|||
|
||||
import org.elasticsearch.action.DocWriteResponse;
|
||||
import org.elasticsearch.action.index.IndexResponse;
|
||||
import org.elasticsearch.action.search.SearchResponse;
|
||||
import org.elasticsearch.action.support.WriteRequest;
|
||||
import org.elasticsearch.common.unit.TimeValue;
|
||||
import org.elasticsearch.search.SearchHit;
|
||||
import org.elasticsearch.search.builder.SearchSourceBuilder;
|
||||
import org.elasticsearch.test.junit.annotations.TestLogging;
|
||||
import org.elasticsearch.xpack.watcher.condition.CompareCondition;
|
||||
import org.elasticsearch.xpack.watcher.execution.ExecutionState;
|
||||
import org.elasticsearch.xpack.watcher.history.HistoryStore;
|
||||
|
@ -20,8 +23,14 @@ import org.joda.time.DateTime;
|
|||
import org.joda.time.DateTimeZone;
|
||||
import org.junit.Before;
|
||||
|
||||
import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.Locale;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static org.elasticsearch.index.query.QueryBuilders.matchQuery;
|
||||
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount;
|
||||
import static org.elasticsearch.xpack.watcher.actions.ActionBuilders.indexAction;
|
||||
import static org.elasticsearch.xpack.watcher.client.WatchSourceBuilders.watchBuilder;
|
||||
import static org.elasticsearch.xpack.watcher.input.InputBuilders.searchInput;
|
||||
|
@ -31,8 +40,13 @@ import static org.elasticsearch.xpack.watcher.trigger.TriggerBuilders.schedule;
|
|||
import static org.elasticsearch.xpack.watcher.trigger.schedule.Schedules.interval;
|
||||
import static org.hamcrest.Matchers.is;
|
||||
|
||||
@TestLogging("org.elasticsearch.xpack.watcher:DEBUG," +
|
||||
"org.elasticsearch.xpack.security.audit.logfile.LoggingAuditTrail:WARN," +
|
||||
"org.elasticsearch.xpack.watcher.WatcherLifeCycleService:DEBUG," +
|
||||
"org.elasticsearch.xpack.watcher.trigger.ScheduleTriggerMock:TRACE," +
|
||||
"org.elasticsearch.xpack.watcher.WatcherIndexingListener:TRACE")
|
||||
public class TimeThrottleIntegrationTests extends AbstractWatcherIntegrationTestCase {
|
||||
|
||||
|
||||
@Override
|
||||
protected boolean timeWarped() {
|
||||
return true;
|
||||
|
@ -62,32 +76,36 @@ public class TimeThrottleIntegrationTests extends AbstractWatcherIntegrationTest
|
|||
|
||||
timeWarp().clock().setTime(DateTime.now(DateTimeZone.UTC));
|
||||
|
||||
timeWarp().scheduler().trigger("_name");
|
||||
timeWarp().trigger("_name");
|
||||
refresh();
|
||||
|
||||
// the first fire should work
|
||||
long actionsCount = docCount("actions", "action", matchAllQuery());
|
||||
assertThat(actionsCount, is(1L));
|
||||
assertHitCount(client().prepareSearch("actions").setTypes("action").get(), 1);
|
||||
|
||||
timeWarp().clock().fastForwardSeconds(5);
|
||||
timeWarp().scheduler().trigger("_name");
|
||||
timeWarp().clock().fastForward(TimeValue.timeValueMillis(4000));
|
||||
timeWarp().trigger("_name");
|
||||
refresh();
|
||||
|
||||
// the last fire should have been throttled, so number of actions shouldn't change
|
||||
actionsCount = docCount("actions", "action", matchAllQuery());
|
||||
assertThat(actionsCount, is(1L));
|
||||
assertHitCount(client().prepareSearch("actions").setTypes("action").get(), 1);
|
||||
|
||||
timeWarp().clock().fastForwardSeconds(30);
|
||||
timeWarp().scheduler().trigger("_name");
|
||||
timeWarp().trigger("_name");
|
||||
refresh();
|
||||
|
||||
// the last fire occurred passed the throttle period, so a new action should have been added
|
||||
actionsCount = docCount("actions", "action", matchAllQuery());
|
||||
assertThat(actionsCount, is(2L));
|
||||
assertHitCount(client().prepareSearch("actions").setTypes("action").get(), 2);
|
||||
|
||||
long throttledCount = docCount(HistoryStore.INDEX_PREFIX_WITH_TEMPLATE + "*", null,
|
||||
matchQuery(WatchRecord.Field.STATE.getPreferredName(), ExecutionState.THROTTLED.id()));
|
||||
assertThat(throttledCount, is(1L));
|
||||
SearchResponse response = client().prepareSearch(HistoryStore.INDEX_PREFIX_WITH_TEMPLATE + "*")
|
||||
.setSource(new SearchSourceBuilder().query(
|
||||
matchQuery(WatchRecord.Field.STATE.getPreferredName(), ExecutionState.THROTTLED.id())))
|
||||
.get();
|
||||
List<Map<String, Object>> hits = Arrays.stream(response.getHits().getHits())
|
||||
.map(SearchHit::getSourceAsMap)
|
||||
.collect(Collectors.toList());
|
||||
|
||||
String message = String.format(Locale.ROOT, "Expected single throttled hits, but was %s", hits);
|
||||
assertThat(message, response.getHits().getTotalHits(), is(1L));
|
||||
}
|
||||
|
||||
public void testTimeThrottleDefaults() throws Exception {
|
||||
|
@ -104,31 +122,32 @@ public class TimeThrottleIntegrationTests extends AbstractWatcherIntegrationTest
|
|||
|
||||
timeWarp().clock().setTime(DateTime.now(DateTimeZone.UTC));
|
||||
|
||||
timeWarp().scheduler().trigger("_name");
|
||||
timeWarp().trigger("_name");
|
||||
refresh();
|
||||
|
||||
// the first trigger should work
|
||||
long actionsCount = docCount("actions", "action", matchAllQuery());
|
||||
assertThat(actionsCount, is(1L));
|
||||
SearchResponse response = client().prepareSearch("actions").setTypes("action").get();
|
||||
assertHitCount(response, 1);
|
||||
|
||||
timeWarp().clock().fastForwardSeconds(2);
|
||||
timeWarp().scheduler().trigger("_name");
|
||||
refresh();
|
||||
timeWarp().trigger("_name");
|
||||
refresh("actions");
|
||||
|
||||
// the last fire should have been throttled, so number of actions shouldn't change
|
||||
actionsCount = docCount("actions", "action", matchAllQuery());
|
||||
assertThat(actionsCount, is(1L));
|
||||
response = client().prepareSearch("actions").setTypes("action").get();
|
||||
assertHitCount(response, 1);
|
||||
|
||||
timeWarp().clock().fastForwardSeconds(10);
|
||||
timeWarp().scheduler().trigger("_name");
|
||||
timeWarp().trigger("_name");
|
||||
refresh();
|
||||
|
||||
// the last fire occurred passed the throttle period, so a new action should have been added
|
||||
actionsCount = docCount("actions", "action", matchAllQuery());
|
||||
assertThat(actionsCount, is(2L));
|
||||
response = client().prepareSearch("actions").setTypes("action").get();
|
||||
assertHitCount(response, 2);
|
||||
|
||||
long throttledCount = docCount(HistoryStore.INDEX_PREFIX_WITH_TEMPLATE + "*", null,
|
||||
matchQuery(WatchRecord.Field.STATE.getPreferredName(), ExecutionState.THROTTLED.id()));
|
||||
assertThat(throttledCount, is(1L));
|
||||
SearchResponse searchResponse = client().prepareSearch(HistoryStore.INDEX_PREFIX_WITH_TEMPLATE + "*")
|
||||
.setQuery(matchQuery(WatchRecord.Field.STATE.getPreferredName(), ExecutionState.THROTTLED.id()))
|
||||
.get();
|
||||
assertHitCount(searchResponse, 1);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -186,7 +186,7 @@ public class EmailAttachmentTests extends AbstractWatcherIntegrationTestCase {
|
|||
.get();
|
||||
|
||||
if (timeWarped()) {
|
||||
timeWarp().scheduler().trigger("_test_id");
|
||||
timeWarp().trigger("_test_id");
|
||||
refresh();
|
||||
}
|
||||
|
||||
|
|
|
@ -82,7 +82,7 @@ public class WebhookHttpsIntegrationTests extends AbstractWatcherIntegrationTest
|
|||
.get();
|
||||
|
||||
if (timeWarped()) {
|
||||
timeWarp().scheduler().trigger("_id");
|
||||
timeWarp().trigger("_id");
|
||||
refresh();
|
||||
}
|
||||
|
||||
|
@ -123,7 +123,7 @@ public class WebhookHttpsIntegrationTests extends AbstractWatcherIntegrationTest
|
|||
.get();
|
||||
|
||||
if (timeWarped()) {
|
||||
timeWarp().scheduler().trigger("_id");
|
||||
timeWarp().trigger("_id");
|
||||
refresh();
|
||||
}
|
||||
|
||||
|
|
|
@ -81,7 +81,7 @@ public class WebhookIntegrationTests extends AbstractWatcherIntegrationTestCase
|
|||
.get();
|
||||
|
||||
if (timeWarped()) {
|
||||
timeWarp().scheduler().trigger("_id");
|
||||
timeWarp().trigger("_id");
|
||||
refresh();
|
||||
}
|
||||
|
||||
|
@ -123,7 +123,7 @@ public class WebhookIntegrationTests extends AbstractWatcherIntegrationTestCase
|
|||
.get();
|
||||
|
||||
if (timeWarped()) {
|
||||
timeWarp().scheduler().trigger("_id");
|
||||
timeWarp().trigger("_id");
|
||||
refresh();
|
||||
}
|
||||
|
||||
|
|
|
@ -8,10 +8,9 @@ package org.elasticsearch.xpack.watcher.execution;
|
|||
import org.elasticsearch.ElasticsearchException;
|
||||
import org.elasticsearch.Version;
|
||||
import org.elasticsearch.action.get.GetResponse;
|
||||
import org.elasticsearch.cluster.ClusterName;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
||||
import org.elasticsearch.cluster.metadata.MetaData;
|
||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||
import org.elasticsearch.cluster.service.ClusterService;
|
||||
import org.elasticsearch.common.collect.Tuple;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.util.concurrent.EsExecutors;
|
||||
|
@ -46,12 +45,14 @@ import org.joda.time.DateTimeZone;
|
|||
import org.junit.Before;
|
||||
|
||||
import java.time.Clock;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.HashSet;
|
||||
import java.util.concurrent.ArrayBlockingQueue;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
|
||||
import static java.util.Arrays.asList;
|
||||
import static java.util.Collections.emptyMap;
|
||||
import static java.util.Collections.singletonMap;
|
||||
import static org.elasticsearch.common.unit.TimeValue.timeValueSeconds;
|
||||
import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
|
||||
|
@ -62,7 +63,6 @@ import static org.hamcrest.Matchers.nullValue;
|
|||
import static org.hamcrest.Matchers.sameInstance;
|
||||
import static org.joda.time.DateTime.now;
|
||||
import static org.mockito.Matchers.any;
|
||||
import static org.mockito.Matchers.anyObject;
|
||||
import static org.mockito.Mockito.doThrow;
|
||||
import static org.mockito.Mockito.eq;
|
||||
import static org.mockito.Mockito.mock;
|
||||
|
@ -106,11 +106,16 @@ public class ExecutionServiceTests extends ESTestCase {
|
|||
|
||||
client = mock(WatcherClientProxy.class);
|
||||
parser = mock(Watch.Parser.class);
|
||||
|
||||
DiscoveryNode discoveryNode = new DiscoveryNode("node_1", ESTestCase.buildNewFakeTransportAddress(), Collections.emptyMap(),
|
||||
new HashSet<>(asList(DiscoveryNode.Role.values())), Version.CURRENT);
|
||||
ClusterService clusterService = mock(ClusterService.class);
|
||||
when(clusterService.localNode()).thenReturn(discoveryNode);
|
||||
|
||||
executionService = new ExecutionService(Settings.EMPTY, historyStore, triggeredWatchStore, executor, clock, threadPool,
|
||||
parser, client);
|
||||
parser, clusterService, client);
|
||||
|
||||
ClusterState clusterState = mock(ClusterState.class);
|
||||
when(triggeredWatchStore.loadTriggeredWatches(clusterState)).thenReturn(new ArrayList<>());
|
||||
executionService.start(clusterState);
|
||||
}
|
||||
|
||||
|
@ -185,6 +190,7 @@ public class ExecutionServiceTests extends ESTestCase {
|
|||
WatchRecord watchRecord = executionService.execute(context);
|
||||
assertThat(watchRecord.result().conditionResult(), sameInstance(conditionResult));
|
||||
assertThat(watchRecord.result().transformResult(), sameInstance(watchTransformResult));
|
||||
assertThat(watchRecord.getNodeId(), is("node_1"));
|
||||
ActionWrapper.Result result = watchRecord.result().actionsResults().get("_action");
|
||||
assertThat(result, notNullValue());
|
||||
assertThat(result.id(), is("_action"));
|
||||
|
@ -809,6 +815,7 @@ public class ExecutionServiceTests extends ESTestCase {
|
|||
when(watch.id()).thenReturn("_id");
|
||||
WatchExecutionContext ctx = mock(WatchExecutionContext.class);
|
||||
when(ctx.knownWatch()).thenReturn(true);
|
||||
when(watch.status()).thenReturn(new WatchStatus(now(), emptyMap()));
|
||||
when(ctx.watch()).thenReturn(watch);
|
||||
|
||||
GetResponse getResponse = mock(GetResponse.class);
|
||||
|
@ -843,18 +850,22 @@ public class ExecutionServiceTests extends ESTestCase {
|
|||
}
|
||||
}
|
||||
|
||||
public void testValidateStartWithClosedIndex() throws Exception {
|
||||
when(triggeredWatchStore.validate(anyObject())).thenReturn(true);
|
||||
ClusterState.Builder csBuilder = new ClusterState.Builder(new ClusterName("_name"));
|
||||
MetaData.Builder metaDataBuilder = MetaData.builder();
|
||||
Settings indexSettings = settings(Version.CURRENT)
|
||||
.put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1)
|
||||
.put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 1)
|
||||
.build();
|
||||
metaDataBuilder.put(IndexMetaData.builder(Watch.INDEX).state(IndexMetaData.State.CLOSE).settings(indexSettings));
|
||||
csBuilder.metaData(metaDataBuilder);
|
||||
public void testWatchInactive() {
|
||||
Watch watch = mock(Watch.class);
|
||||
when(watch.id()).thenReturn("_id");
|
||||
WatchExecutionContext ctx = mock(WatchExecutionContext.class);
|
||||
when(ctx.knownWatch()).thenReturn(true);
|
||||
WatchStatus status = mock(WatchStatus.class);
|
||||
when(status.state()).thenReturn(new WatchStatus.State(false, now()));
|
||||
when(watch.status()).thenReturn(status);
|
||||
when(ctx.watch()).thenReturn(watch);
|
||||
|
||||
assertThat(executionService.validate(csBuilder.build()), is(false));
|
||||
WatchRecord.MessageWatchRecord record = mock(WatchRecord.MessageWatchRecord.class);
|
||||
when(record.state()).thenReturn(ExecutionState.EXECUTION_NOT_NEEDED);
|
||||
when(ctx.abortBeforeExecution(eq(ExecutionState.EXECUTION_NOT_NEEDED), eq("Watch is not active"))).thenReturn(record);
|
||||
|
||||
WatchRecord watchRecord = executionService.execute(ctx);
|
||||
assertThat(watchRecord.state(), is(ExecutionState.EXECUTION_NOT_NEEDED));
|
||||
}
|
||||
|
||||
private Tuple<Condition, Condition.Result> whenCondition(final WatchExecutionContext context) {
|
||||
|
|
|
@ -1,69 +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.xpack.watcher.execution;
|
||||
|
||||
import org.elasticsearch.action.get.GetResponse;
|
||||
import org.elasticsearch.cluster.service.ClusterService;
|
||||
import org.elasticsearch.xpack.watcher.condition.Condition;
|
||||
import org.elasticsearch.xpack.watcher.condition.AlwaysCondition;
|
||||
import org.elasticsearch.xpack.watcher.input.none.ExecutableNoneInput;
|
||||
import org.elasticsearch.xpack.watcher.test.AbstractWatcherIntegrationTestCase;
|
||||
import org.elasticsearch.xpack.watcher.trigger.schedule.ScheduleTriggerEvent;
|
||||
import org.elasticsearch.xpack.watcher.watch.Watch;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.DateTimeZone;
|
||||
|
||||
import java.util.Collection;
|
||||
|
||||
import static org.hamcrest.Matchers.equalTo;
|
||||
import static org.hamcrest.Matchers.hasSize;
|
||||
import static org.hamcrest.Matchers.is;
|
||||
import static org.hamcrest.Matchers.notNullValue;
|
||||
|
||||
public class TriggeredWatchStoreLifeCycleTests extends AbstractWatcherIntegrationTestCase {
|
||||
public void testPutLoadUpdate() throws Exception {
|
||||
Condition condition = AlwaysCondition.INSTANCE;
|
||||
TriggeredWatchStore triggeredWatchStore = getInstanceFromMaster(TriggeredWatchStore.class);
|
||||
Watch watch = new Watch("_name", null, new ExecutableNoneInput(logger), condition, null, null, null, null, null);
|
||||
|
||||
// Put watch records and verify that these are stored
|
||||
TriggeredWatch[] triggeredWatches = new TriggeredWatch[randomIntBetween(1, 50)];
|
||||
for (int i = 0; i < triggeredWatches.length; i++) {
|
||||
DateTime dateTime = new DateTime(i, DateTimeZone.UTC);
|
||||
ScheduleTriggerEvent event = new ScheduleTriggerEvent(watch.id(), dateTime, dateTime);
|
||||
Wid wid = new Wid("record_" + i, DateTime.now(DateTimeZone.UTC));
|
||||
triggeredWatches[i] = new TriggeredWatch(wid, event);
|
||||
triggeredWatchStore.put(triggeredWatches[i]);
|
||||
GetResponse getResponse = client().prepareGet(TriggeredWatchStore.INDEX_NAME, TriggeredWatchStore.DOC_TYPE,
|
||||
triggeredWatches[i].id().value())
|
||||
.setVersion(1)
|
||||
.get();
|
||||
assertThat(getResponse.isExists(), equalTo(true));
|
||||
}
|
||||
|
||||
// Load the stored watch records
|
||||
ClusterService clusterService = getInstanceFromMaster(ClusterService.class);
|
||||
Collection<TriggeredWatch> loadedTriggeredWatches = triggeredWatchStore.loadTriggeredWatches(clusterService.state());
|
||||
assertThat(loadedTriggeredWatches, notNullValue());
|
||||
assertThat(loadedTriggeredWatches, hasSize(triggeredWatches.length));
|
||||
|
||||
// Change the state to executed and update the watch records and then verify if the changes have been persisted too
|
||||
for (TriggeredWatch triggeredWatch : triggeredWatches) {
|
||||
assertThat(loadedTriggeredWatches.contains(triggeredWatch), is(true));
|
||||
triggeredWatchStore.delete(triggeredWatch.id());
|
||||
GetResponse getResponse = client().prepareGet(TriggeredWatchStore.INDEX_NAME, TriggeredWatchStore.DOC_TYPE,
|
||||
triggeredWatch.id().value())
|
||||
.setVersion(2L)
|
||||
.get();
|
||||
assertThat(getResponse.isExists(), equalTo(false));
|
||||
}
|
||||
|
||||
// try to load watch records, but none are in the await state, so no watch records are loaded.
|
||||
loadedTriggeredWatches = triggeredWatchStore.loadTriggeredWatches(clusterService.state());
|
||||
assertThat(loadedTriggeredWatches, notNullValue());
|
||||
assertThat(loadedTriggeredWatches, hasSize(0));
|
||||
}
|
||||
}
|
|
@ -5,6 +5,7 @@
|
|||
*/
|
||||
package org.elasticsearch.xpack.watcher.execution;
|
||||
|
||||
import org.elasticsearch.ElasticsearchException;
|
||||
import org.elasticsearch.Version;
|
||||
import org.elasticsearch.action.admin.indices.refresh.RefreshRequest;
|
||||
import org.elasticsearch.action.admin.indices.refresh.RefreshResponse;
|
||||
|
@ -35,16 +36,23 @@ import org.elasticsearch.search.SearchHits;
|
|||
import org.elasticsearch.search.internal.InternalSearchResponse;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
import org.elasticsearch.xpack.watcher.support.init.proxy.WatcherClientProxy;
|
||||
import org.elasticsearch.xpack.watcher.trigger.schedule.ScheduleTriggerEvent;
|
||||
import org.elasticsearch.xpack.watcher.watch.Watch;
|
||||
import org.hamcrest.core.IsNull;
|
||||
import org.joda.time.DateTime;
|
||||
import org.junit.Before;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
|
||||
import static org.hamcrest.Matchers.hasSize;
|
||||
import static org.hamcrest.Matchers.is;
|
||||
import static org.hamcrest.Matchers.notNullValue;
|
||||
import static org.hamcrest.core.IsEqual.equalTo;
|
||||
import static org.joda.time.DateTimeZone.UTC;
|
||||
import static org.mockito.Matchers.any;
|
||||
import static org.mockito.Matchers.anyLong;
|
||||
import static org.mockito.Matchers.anyString;
|
||||
import static org.mockito.Matchers.eq;
|
||||
import static org.mockito.Mockito.mock;
|
||||
|
@ -72,20 +80,17 @@ public class TriggeredWatchStoreTests extends ESTestCase {
|
|||
triggeredWatchStore.start();
|
||||
}
|
||||
|
||||
public void testLoadWatchRecordsNoPriorHistoryIndices() throws Exception {
|
||||
ClusterState.Builder csBuilder = new ClusterState.Builder(new ClusterName("name"));
|
||||
MetaData.Builder metaDataBuilder = MetaData.builder();
|
||||
csBuilder.metaData(metaDataBuilder);
|
||||
ClusterState cs = csBuilder.build();
|
||||
|
||||
assertThat(triggeredWatchStore.validate(cs), is(true));
|
||||
Collection<TriggeredWatch> records = triggeredWatchStore.loadTriggeredWatches(cs);
|
||||
assertThat(records, notNullValue());
|
||||
assertThat(records, hasSize(0));
|
||||
verifyZeroInteractions(clientProxy);
|
||||
public void testFindTriggeredWatchesEmptyCollection() throws Exception {
|
||||
Collection<TriggeredWatch> triggeredWatches = triggeredWatchStore.findTriggeredWatches(Collections.emptyList());
|
||||
assertThat(triggeredWatches, hasSize(0));
|
||||
}
|
||||
|
||||
public void testLoadWatchRecordsNoActivePrimaryShards() throws Exception {
|
||||
public void testValidateNoIndex() {
|
||||
ClusterState.Builder csBuilder = new ClusterState.Builder(new ClusterName("name"));
|
||||
assertThat(triggeredWatchStore.validate(csBuilder.build()), is(true));
|
||||
}
|
||||
|
||||
public void testValidateNoActivePrimaryShards() throws Exception {
|
||||
ClusterState.Builder csBuilder = new ClusterState.Builder(new ClusterName("name"));
|
||||
|
||||
RoutingTable.Builder routingTableBuilder = RoutingTable.builder();
|
||||
|
@ -122,123 +127,9 @@ public class TriggeredWatchStoreTests extends ESTestCase {
|
|||
ClusterState cs = csBuilder.build();
|
||||
|
||||
assertThat(triggeredWatchStore.validate(cs), is(false));
|
||||
IllegalStateException e = expectThrows(IllegalStateException.class, () -> triggeredWatchStore.loadTriggeredWatches(cs));
|
||||
assertThat(e.getMessage(), is("not all primary shards of the triggered watches index [.triggered_watches] are started"));
|
||||
|
||||
verifyZeroInteractions(clientProxy);
|
||||
}
|
||||
|
||||
public void testLoadWatchRecordsRefreshNotHittingAllShards() throws Exception {
|
||||
ClusterState.Builder csBuilder = new ClusterState.Builder(new ClusterName("_name"));
|
||||
|
||||
RoutingTable.Builder routingTableBuilder = RoutingTable.builder();
|
||||
MetaData.Builder metaDataBuilder = MetaData.builder();
|
||||
metaDataBuilder.put(IndexMetaData.builder(TriggeredWatchStore.INDEX_NAME).settings(indexSettings));
|
||||
final Index index = metaDataBuilder.get(TriggeredWatchStore.INDEX_NAME).getIndex();
|
||||
IndexRoutingTable.Builder indexRoutingTableBuilder = IndexRoutingTable.builder(index);
|
||||
ShardId shardId = new ShardId(index, 0);
|
||||
indexRoutingTableBuilder.addIndexShard(new IndexShardRoutingTable.Builder(shardId)
|
||||
.addShard(TestShardRouting.newShardRouting(shardId, "_node_id", null, true, ShardRoutingState.STARTED))
|
||||
.build());
|
||||
indexRoutingTableBuilder.addReplica();
|
||||
routingTableBuilder.add(indexRoutingTableBuilder.build());
|
||||
csBuilder.metaData(metaDataBuilder);
|
||||
csBuilder.routingTable(routingTableBuilder.build());
|
||||
ClusterState cs = csBuilder.build();
|
||||
|
||||
assertThat(triggeredWatchStore.validate(cs), is(true));
|
||||
RefreshResponse refreshResponse = mockRefreshResponse(1, 0);
|
||||
when(clientProxy.refresh(any(RefreshRequest.class))).thenReturn(refreshResponse);
|
||||
try {
|
||||
triggeredWatchStore.loadTriggeredWatches(cs);
|
||||
fail("exception expected, because refresh did't manage to run on all primary shards");
|
||||
} catch (Exception e) {
|
||||
assertThat(e.getMessage(), equalTo("refresh was supposed to run on [1] shards, but ran on [0] shards"));
|
||||
}
|
||||
|
||||
verify(clientProxy, times(1)).refresh(any(RefreshRequest.class));
|
||||
}
|
||||
|
||||
public void testLoadWatchRecordsSearchNotHittingAllShards() throws Exception {
|
||||
ClusterState.Builder csBuilder = new ClusterState.Builder(new ClusterName("_name"));
|
||||
|
||||
RoutingTable.Builder routingTableBuilder = RoutingTable.builder();
|
||||
MetaData.Builder metaDataBuilder = MetaData.builder();
|
||||
metaDataBuilder.put(IndexMetaData.builder(TriggeredWatchStore.INDEX_NAME).settings(indexSettings));
|
||||
final Index index = metaDataBuilder.get(TriggeredWatchStore.INDEX_NAME).getIndex();
|
||||
IndexRoutingTable.Builder indexRoutingTableBuilder = IndexRoutingTable.builder(index);
|
||||
ShardId shardId = new ShardId(index, 0);
|
||||
indexRoutingTableBuilder.addIndexShard(new IndexShardRoutingTable.Builder(shardId)
|
||||
.addShard(TestShardRouting.newShardRouting(shardId, "_node_name", null, true, ShardRoutingState.STARTED))
|
||||
.build());
|
||||
indexRoutingTableBuilder.addReplica();
|
||||
routingTableBuilder.add(indexRoutingTableBuilder.build());
|
||||
csBuilder.metaData(metaDataBuilder);
|
||||
csBuilder.routingTable(routingTableBuilder.build());
|
||||
ClusterState cs = csBuilder.build();
|
||||
|
||||
RefreshResponse refreshResponse = mockRefreshResponse(1, 1);
|
||||
when(clientProxy.refresh(any(RefreshRequest.class))).thenReturn(refreshResponse);
|
||||
|
||||
SearchResponse searchResponse = mock(SearchResponse.class);
|
||||
when(searchResponse.getSuccessfulShards()).thenReturn(0);
|
||||
when(searchResponse.getTotalShards()).thenReturn(1);
|
||||
when(clientProxy.search(any(SearchRequest.class), any(TimeValue.class))).thenReturn(searchResponse);
|
||||
|
||||
when(clientProxy.clearScroll(anyString())).thenReturn(new ClearScrollResponse(true, 1));
|
||||
|
||||
assertThat(triggeredWatchStore.validate(cs), is(true));
|
||||
try {
|
||||
triggeredWatchStore.loadTriggeredWatches(cs);
|
||||
fail("exception expected, because scan search didn't manage to run on all shards");
|
||||
} catch (Exception e) {
|
||||
assertThat(e.getMessage(), equalTo("scan search was supposed to run on [1] shards, but ran on [0] shards"));
|
||||
}
|
||||
verify(clientProxy, times(1)).refresh(any(RefreshRequest.class));
|
||||
verify(clientProxy, times(1)).search(any(SearchRequest.class), any(TimeValue.class));
|
||||
verify(clientProxy, times(1)).clearScroll(anyString());
|
||||
}
|
||||
|
||||
public void testLoadWatchRecordsNoHistoryEntries() throws Exception {
|
||||
ClusterState.Builder csBuilder = new ClusterState.Builder(new ClusterName("_name"));
|
||||
|
||||
RoutingTable.Builder routingTableBuilder = RoutingTable.builder();
|
||||
MetaData.Builder metaDataBuilder = MetaData.builder();
|
||||
metaDataBuilder.put(IndexMetaData.builder(TriggeredWatchStore.INDEX_NAME).settings(indexSettings));
|
||||
final Index index = metaDataBuilder.get(TriggeredWatchStore.INDEX_NAME).getIndex();
|
||||
IndexRoutingTable.Builder indexRoutingTableBuilder = IndexRoutingTable.builder(index);
|
||||
ShardId shardId = new ShardId(index, 0);
|
||||
indexRoutingTableBuilder.addIndexShard(new IndexShardRoutingTable.Builder(shardId)
|
||||
.addShard(TestShardRouting.newShardRouting(shardId, "_node_name", null, true, ShardRoutingState.STARTED))
|
||||
.build());
|
||||
indexRoutingTableBuilder.addReplica();
|
||||
routingTableBuilder.add(indexRoutingTableBuilder.build());
|
||||
csBuilder.metaData(metaDataBuilder);
|
||||
csBuilder.routingTable(routingTableBuilder.build());
|
||||
ClusterState cs = csBuilder.build();
|
||||
|
||||
RefreshResponse refreshResponse = mockRefreshResponse(1, 1);
|
||||
when(clientProxy.refresh(any(RefreshRequest.class))).thenReturn(refreshResponse);
|
||||
|
||||
SearchResponse searchResponse = mock(SearchResponse.class);
|
||||
when(searchResponse.getSuccessfulShards()).thenReturn(1);
|
||||
when(searchResponse.getTotalShards()).thenReturn(1);
|
||||
when(searchResponse.getHits()).thenReturn(SearchHits.empty());
|
||||
when(clientProxy.search(any(SearchRequest.class), any(TimeValue.class))).thenReturn(searchResponse);
|
||||
|
||||
when(clientProxy.clearScroll(anyString())).thenReturn(new ClearScrollResponse(true, 1));
|
||||
|
||||
assertThat(triggeredWatchStore.validate(cs), is(true));
|
||||
Collection<TriggeredWatch> triggeredWatches = triggeredWatchStore.loadTriggeredWatches(cs);
|
||||
assertThat(triggeredWatches, IsNull.notNullValue());
|
||||
assertThat(triggeredWatches, hasSize(0));
|
||||
|
||||
verify(clientProxy, times(1)).refresh(any(RefreshRequest.class));
|
||||
verify(clientProxy, times(1)).search(any(SearchRequest.class), any(TimeValue.class));
|
||||
verify(clientProxy, times(1)).clearScroll(anyString());
|
||||
}
|
||||
|
||||
public void testLoadWatchRecordsFoundHistoryEntries() throws Exception {
|
||||
public void testFindTriggeredWatchesGoodCase() throws Exception {
|
||||
ClusterState.Builder csBuilder = new ClusterState.Builder(new ClusterName("_name"));
|
||||
|
||||
RoutingTable.Builder routingTableBuilder = RoutingTable.builder();
|
||||
|
@ -262,20 +153,21 @@ public class TriggeredWatchStoreTests extends ESTestCase {
|
|||
SearchResponse searchResponse1 = mock(SearchResponse.class);
|
||||
when(searchResponse1.getSuccessfulShards()).thenReturn(1);
|
||||
when(searchResponse1.getTotalShards()).thenReturn(1);
|
||||
SearchHit hit = new SearchHit(0, "_id", new Text("_type"), null);
|
||||
BytesArray source = new BytesArray("{}");
|
||||
SearchHit hit = new SearchHit(0, "first_foo", new Text(TriggeredWatchStore.DOC_TYPE), null);
|
||||
hit.version(1L);
|
||||
hit.shard(new SearchShardTarget("_node_id", index, 0));
|
||||
hit.sourceRef(new BytesArray("{}"));
|
||||
hit.sourceRef(source);
|
||||
SearchHits hits = new SearchHits(new SearchHit[]{hit}, 1, 1.0f);
|
||||
when(searchResponse1.getHits()).thenReturn(hits);
|
||||
when(searchResponse1.getScrollId()).thenReturn("_scrollId");
|
||||
when(clientProxy.search(any(SearchRequest.class), any(TimeValue.class))).thenReturn(searchResponse1);
|
||||
when(clientProxy.search(any(SearchRequest.class))).thenReturn(searchResponse1);
|
||||
|
||||
// First return a scroll response with a single hit and then with no hits
|
||||
hit = new SearchHit(0, "_id", new Text("_type"), null);
|
||||
hit = new SearchHit(0, "second_foo", new Text(TriggeredWatchStore.DOC_TYPE), null);
|
||||
hit.version(1L);
|
||||
hit.shard(new SearchShardTarget("_node_id", index, 0));
|
||||
hit.sourceRef(new BytesArray("{}"));
|
||||
hit.sourceRef(source);
|
||||
hits = new SearchHits(new SearchHit[]{hit}, 1, 1.0f);
|
||||
SearchResponse searchResponse2 = new SearchResponse(
|
||||
new InternalSearchResponse(hits, null, null, null, false, null, 1), "_scrollId", 1, 1, 1, null);
|
||||
|
@ -289,12 +181,30 @@ public class TriggeredWatchStoreTests extends ESTestCase {
|
|||
when(clientProxy.clearScroll(anyString())).thenReturn(new ClearScrollResponse(true, 1));
|
||||
|
||||
assertThat(triggeredWatchStore.validate(cs), is(true));
|
||||
Collection<TriggeredWatch> triggeredWatches = triggeredWatchStore.loadTriggeredWatches(cs);
|
||||
DateTime now = DateTime.now(UTC);
|
||||
ScheduleTriggerEvent triggerEvent = new ScheduleTriggerEvent(now, now);
|
||||
|
||||
Watch watch1 = mock(Watch.class);
|
||||
when(watch1.id()).thenReturn("first");
|
||||
TriggeredWatch triggeredWatch1 = new TriggeredWatch(new Wid("first", now), triggerEvent);
|
||||
when(parser.parse(eq("first_foo"), anyLong(), eq(source))).thenReturn(triggeredWatch1);
|
||||
|
||||
Watch watch2 = mock(Watch.class);
|
||||
when(watch2.id()).thenReturn("second");
|
||||
TriggeredWatch triggeredWatch2 = new TriggeredWatch(new Wid("second", now), triggerEvent);
|
||||
when(parser.parse(eq("second_foo"), anyLong(), eq(source))).thenReturn(triggeredWatch2);
|
||||
|
||||
Collection<Watch> watches = new ArrayList<>();
|
||||
watches.add(watch1);
|
||||
if (randomBoolean()) {
|
||||
watches.add(watch2);
|
||||
}
|
||||
Collection<TriggeredWatch> triggeredWatches = triggeredWatchStore.findTriggeredWatches(watches);
|
||||
assertThat(triggeredWatches, notNullValue());
|
||||
assertThat(triggeredWatches, hasSize(2));
|
||||
assertThat(triggeredWatches, hasSize(watches.size()));
|
||||
|
||||
verify(clientProxy, times(1)).refresh(any(RefreshRequest.class));
|
||||
verify(clientProxy, times(1)).search(any(SearchRequest.class), any(TimeValue.class));
|
||||
verify(clientProxy, times(1)).search(any(SearchRequest.class));
|
||||
verify(clientProxy, times(2)).searchScroll(anyString(), any(TimeValue.class));
|
||||
verify(clientProxy, times(1)).clearScroll(anyString());
|
||||
}
|
||||
|
@ -321,7 +231,6 @@ public class TriggeredWatchStoreTests extends ESTestCase {
|
|||
ClusterState cs = csBuilder.build();
|
||||
|
||||
assertThat(triggeredWatchStore.validate(cs), is(true));
|
||||
verifyZeroInteractions(clientProxy);
|
||||
}
|
||||
|
||||
// the elasticsearch migration helper is doing reindex using aliases, so we have to
|
||||
|
@ -353,7 +262,6 @@ public class TriggeredWatchStoreTests extends ESTestCase {
|
|||
ClusterState cs = csBuilder.build();
|
||||
|
||||
assertThat(triggeredWatchStore.validate(cs), is(false));
|
||||
verifyZeroInteractions(clientProxy);
|
||||
}
|
||||
|
||||
// this is a special condition that could lead to an NPE in earlier versions
|
||||
|
|
|
@ -12,6 +12,7 @@ import org.elasticsearch.script.Script;
|
|||
import org.elasticsearch.script.ScriptType;
|
||||
import org.elasticsearch.search.SearchHit;
|
||||
import org.elasticsearch.search.builder.SearchSourceBuilder;
|
||||
import org.elasticsearch.test.junit.annotations.TestLogging;
|
||||
import org.elasticsearch.xpack.watcher.client.WatchSourceBuilder;
|
||||
import org.elasticsearch.xpack.watcher.condition.AlwaysCondition;
|
||||
import org.elasticsearch.xpack.watcher.condition.CompareCondition;
|
||||
|
@ -43,6 +44,7 @@ import static org.hamcrest.Matchers.nullValue;
|
|||
/**
|
||||
* This test makes sure per-action conditions are honored.
|
||||
*/
|
||||
@TestLogging("org.elasticsearch.xpack.watcher:DEBUG,org.elasticsearch.xpack.watcher.WatcherIndexingListener:TRACE")
|
||||
public class HistoryActionConditionTests extends AbstractWatcherIntegrationTestCase {
|
||||
|
||||
private final Input input = simpleInput("key", 15).build();
|
||||
|
@ -143,16 +145,12 @@ public class HistoryActionConditionTests extends AbstractWatcherIntegrationTestC
|
|||
@SuppressWarnings("unchecked")
|
||||
public void testActionConditionWithFailures() throws Exception {
|
||||
final String id = "testActionConditionWithFailures";
|
||||
final List<Condition> actionConditionsWithFailure = Arrays.asList(conditionFails, conditionPasses, AlwaysCondition.INSTANCE);
|
||||
final Condition[] actionConditionsWithFailure = new Condition[] { conditionFails, conditionPasses, AlwaysCondition.INSTANCE };
|
||||
Collections.shuffle(Arrays.asList(actionConditionsWithFailure), random());
|
||||
|
||||
Collections.shuffle(actionConditionsWithFailure, random());
|
||||
|
||||
final int failedIndex = actionConditionsWithFailure.indexOf(conditionFails);
|
||||
|
||||
putAndTriggerWatch(id, input, actionConditionsWithFailure.toArray(new Condition[actionConditionsWithFailure.size()]));
|
||||
|
||||
flush();
|
||||
final int failedIndex = Arrays.asList(actionConditionsWithFailure).indexOf(conditionFails);
|
||||
|
||||
putAndTriggerWatch(id, input, actionConditionsWithFailure);
|
||||
assertWatchWithMinimumActionsCount(id, ExecutionState.EXECUTED, 1);
|
||||
|
||||
// only one action should have failed via condition
|
||||
|
@ -162,13 +160,13 @@ public class HistoryActionConditionTests extends AbstractWatcherIntegrationTestC
|
|||
final SearchHit hit = response.getHits().getAt(0);
|
||||
final List<Object> actions = getActionsFromHit(hit.getSourceAsMap());
|
||||
|
||||
for (int i = 0; i < actionConditionsWithFailure.size(); ++i) {
|
||||
for (int i = 0; i < actionConditionsWithFailure.length; ++i) {
|
||||
final Map<String, Object> action = (Map<String, Object>)actions.get(i);
|
||||
final Map<String, Object> condition = (Map<String, Object>)action.get("condition");
|
||||
final Map<String, Object> logging = (Map<String, Object>)action.get("logging");
|
||||
|
||||
assertThat(action.get("id"), is("action" + i));
|
||||
assertThat(condition.get("type"), is(actionConditionsWithFailure.get(i).type()));
|
||||
assertThat(condition.get("type"), is(actionConditionsWithFailure[i].type()));
|
||||
|
||||
if (i == failedIndex) {
|
||||
assertThat(action.get("status"), is("condition_failed"));
|
||||
|
@ -188,13 +186,16 @@ public class HistoryActionConditionTests extends AbstractWatcherIntegrationTestC
|
|||
public void testActionCondition() throws Exception {
|
||||
final String id = "testActionCondition";
|
||||
final List<Condition> actionConditions = new ArrayList<>();
|
||||
actionConditions.add(conditionPasses);
|
||||
//actionConditions.add(conditionPasses);
|
||||
actionConditions.add(AlwaysCondition.INSTANCE);
|
||||
|
||||
/*
|
||||
if (randomBoolean()) {
|
||||
actionConditions.add(AlwaysCondition.INSTANCE);
|
||||
}
|
||||
|
||||
Collections.shuffle(actionConditions, random());
|
||||
*/
|
||||
|
||||
putAndTriggerWatch(id, input, actionConditions.toArray(new Condition[actionConditions.size()]));
|
||||
|
||||
|
@ -256,7 +257,7 @@ public class HistoryActionConditionTests extends AbstractWatcherIntegrationTestC
|
|||
|
||||
assertThat(putWatchResponse.isCreated(), is(true));
|
||||
|
||||
timeWarp().scheduler().trigger(id);
|
||||
timeWarp().trigger(id);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -50,6 +50,7 @@ import static org.mockito.Mockito.verify;
|
|||
import static org.mockito.Mockito.when;
|
||||
|
||||
public class HistoryStoreTests extends ESTestCase {
|
||||
|
||||
private HistoryStore historyStore;
|
||||
private WatcherClientProxy clientProxy;
|
||||
|
||||
|
@ -63,7 +64,7 @@ public class HistoryStoreTests extends ESTestCase {
|
|||
public void testPut() throws Exception {
|
||||
Wid wid = new Wid("_name", new DateTime(0, UTC));
|
||||
ScheduleTriggerEvent event = new ScheduleTriggerEvent(wid.watchId(), new DateTime(0, UTC), new DateTime(0, UTC));
|
||||
WatchRecord watchRecord = new WatchRecord.MessageWatchRecord(wid, event, ExecutionState.EXECUTED, null);
|
||||
WatchRecord watchRecord = new WatchRecord.MessageWatchRecord(wid, event, ExecutionState.EXECUTED, null, randomAlphaOfLength(10));
|
||||
|
||||
IndexResponse indexResponse = mock(IndexResponse.class);
|
||||
IndexRequest indexRequest = indexRequest(".watcher-history-1970.01.01", HistoryStore.DOC_TYPE, wid.value()
|
||||
|
@ -76,7 +77,7 @@ public class HistoryStoreTests extends ESTestCase {
|
|||
public void testPutStopped() throws Exception {
|
||||
Wid wid = new Wid("_name", new DateTime(0, UTC));
|
||||
ScheduleTriggerEvent event = new ScheduleTriggerEvent(wid.watchId(), new DateTime(0, UTC), new DateTime(0, UTC));
|
||||
WatchRecord watchRecord = new WatchRecord.MessageWatchRecord(wid, event, ExecutionState.EXECUTED, null);
|
||||
WatchRecord watchRecord = new WatchRecord.MessageWatchRecord(wid, event, ExecutionState.EXECUTED, null, randomAlphaOfLength(10));
|
||||
|
||||
historyStore.stop();
|
||||
try {
|
||||
|
|
|
@ -9,6 +9,7 @@ import org.elasticsearch.action.search.SearchResponse;
|
|||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.search.aggregations.Aggregations;
|
||||
import org.elasticsearch.search.aggregations.bucket.terms.Terms;
|
||||
import org.elasticsearch.test.junit.annotations.TestLogging;
|
||||
import org.elasticsearch.xpack.notification.email.EmailTemplate;
|
||||
import org.elasticsearch.xpack.notification.email.support.EmailServer;
|
||||
import org.elasticsearch.xpack.watcher.condition.AlwaysCondition;
|
||||
|
@ -31,6 +32,8 @@ import static org.hamcrest.Matchers.notNullValue;
|
|||
* This test makes sure that the email address fields in the watch_record action result are
|
||||
* not analyzed so they can be used in aggregations
|
||||
*/
|
||||
@TestLogging("org.elasticsearch.xpack.watcher:DEBUG," +
|
||||
"org.elasticsearch.xpack.watcher.WatcherIndexingListener:TRACE")
|
||||
public class HistoryTemplateEmailMappingsTests extends AbstractWatcherIntegrationTestCase {
|
||||
|
||||
private EmailServer server;
|
||||
|
@ -82,7 +85,7 @@ public class HistoryTemplateEmailMappingsTests extends AbstractWatcherIntegratio
|
|||
.get();
|
||||
|
||||
assertThat(putWatchResponse.isCreated(), is(true));
|
||||
timeWarp().scheduler().trigger("_id");
|
||||
timeWarp().trigger("_id");
|
||||
flush();
|
||||
refresh();
|
||||
|
||||
|
|
|
@ -77,7 +77,7 @@ public class HistoryTemplateHttpMappingsTests extends AbstractWatcherIntegration
|
|||
webServer.enqueue(new MockResponse().setResponseCode(200).setBody("{}"));
|
||||
|
||||
assertThat(putWatchResponse.isCreated(), is(true));
|
||||
timeWarp().scheduler().trigger("_id");
|
||||
timeWarp().trigger("_id");
|
||||
flush();
|
||||
refresh();
|
||||
|
||||
|
|
|
@ -41,12 +41,12 @@ public class HistoryTemplateIndexActionMappingsTests extends AbstractWatcherInte
|
|||
String type = "the-type";
|
||||
|
||||
PutWatchResponse putWatchResponse = watcherClient().preparePutWatch("_id").setSource(watchBuilder()
|
||||
.trigger(schedule(interval("5s")))
|
||||
.trigger(schedule(interval("5m")))
|
||||
.addAction("index", indexAction(index, type)))
|
||||
.get();
|
||||
|
||||
assertThat(putWatchResponse.isCreated(), is(true));
|
||||
timeWarp().scheduler().trigger("_id");
|
||||
timeWarp().trigger("_id");
|
||||
flush();
|
||||
refresh();
|
||||
|
||||
|
|
|
@ -62,7 +62,7 @@ public class HistoryTemplateSearchInputMappingsTests extends AbstractWatcherInte
|
|||
.get();
|
||||
|
||||
assertThat(putWatchResponse.isCreated(), is(true));
|
||||
timeWarp().scheduler().trigger("_id");
|
||||
timeWarp().trigger("_id");
|
||||
flush();
|
||||
refresh();
|
||||
|
||||
|
|
|
@ -49,36 +49,29 @@ public class HistoryTemplateTimeMappingsTests extends AbstractWatcherIntegration
|
|||
.get();
|
||||
|
||||
assertThat(putWatchResponse.isCreated(), is(true));
|
||||
timeWarp().scheduler().trigger("_id");
|
||||
flush();
|
||||
refresh();
|
||||
timeWarp().trigger("_id");
|
||||
|
||||
// the action should fail as no email server is available
|
||||
assertWatchWithMinimumActionsCount("_id", ExecutionState.EXECUTED, 1);
|
||||
refresh();
|
||||
assertBusy(new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
GetMappingsResponse mappingsResponse = client().admin().indices().prepareGetMappings().get();
|
||||
assertThat(mappingsResponse, notNullValue());
|
||||
assertThat(mappingsResponse.getMappings().isEmpty(), is(false));
|
||||
for (ObjectObjectCursor<String, ImmutableOpenMap<String, MappingMetaData>> metadatas : mappingsResponse.getMappings()) {
|
||||
if (!metadatas.key.startsWith(".watcher-history")) {
|
||||
continue;
|
||||
}
|
||||
MappingMetaData metadata = metadatas.value.get("watch_record");
|
||||
assertThat(metadata, notNullValue());
|
||||
try {
|
||||
Map<String, Object> source = metadata.getSourceAsMap();
|
||||
logger.info("checking index [{}] with metadata:\n[{}]", metadatas.key, metadata.source().toString());
|
||||
assertThat(extractValue("properties.trigger_event.properties.type.type", source), is((Object) "keyword"));
|
||||
assertThat(extractValue("properties.trigger_event.properties.triggered_time.type", source), is((Object) "date"));
|
||||
assertThat(extractValue("properties.trigger_event.properties.schedule.properties.scheduled_time.type", source),
|
||||
is((Object) "date"));
|
||||
assertThat(extractValue("properties.result.properties.execution_time.type", source), is((Object) "date"));
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
assertBusy(() -> {
|
||||
GetMappingsResponse mappingsResponse = client().admin().indices().prepareGetMappings().get();
|
||||
assertThat(mappingsResponse, notNullValue());
|
||||
assertThat(mappingsResponse.getMappings().isEmpty(), is(false));
|
||||
for (ObjectObjectCursor<String, ImmutableOpenMap<String, MappingMetaData>> metadatas : mappingsResponse.getMappings()) {
|
||||
if (!metadatas.key.startsWith(HistoryStore.INDEX_PREFIX)) {
|
||||
continue;
|
||||
}
|
||||
MappingMetaData metadata = metadatas.value.get("watch_record");
|
||||
assertThat(metadata, notNullValue());
|
||||
try {
|
||||
Map<String, Object> source = metadata.getSourceAsMap();
|
||||
logger.info("checking index [{}] with metadata:\n[{}]", metadatas.key, metadata.source().toString());
|
||||
assertThat(extractValue("properties.trigger_event.properties.type.type", source), is((Object) "keyword"));
|
||||
assertThat(extractValue("properties.trigger_event.properties.triggered_time.type", source), is((Object) "date"));
|
||||
assertThat(extractValue("properties.trigger_event.properties.schedule.properties.scheduled_time.type", source),
|
||||
is((Object) "date"));
|
||||
assertThat(extractValue("properties.result.properties.execution_time.type", source), is((Object) "date"));
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
});
|
||||
|
|
|
@ -8,6 +8,7 @@ package org.elasticsearch.xpack.watcher.history;
|
|||
import org.elasticsearch.action.admin.indices.mapping.get.GetFieldMappingsResponse;
|
||||
import org.elasticsearch.plugins.Plugin;
|
||||
import org.elasticsearch.script.MockScriptPlugin;
|
||||
import org.elasticsearch.test.junit.annotations.TestLogging;
|
||||
import org.elasticsearch.xpack.watcher.condition.AlwaysCondition;
|
||||
import org.elasticsearch.xpack.watcher.execution.ExecutionState;
|
||||
import org.elasticsearch.xpack.watcher.test.AbstractWatcherIntegrationTestCase;
|
||||
|
@ -32,6 +33,7 @@ import static org.hamcrest.Matchers.is;
|
|||
* This test makes sure that the http host and path fields in the watch_record action result are
|
||||
* not analyzed so they can be used in aggregations
|
||||
*/
|
||||
@TestLogging("org.elasticsearch.xpack.watcher:DEBUG,org.elasticsearch.xpack.watcher.WatcherIndexingListener:TRACE")
|
||||
public class HistoryTemplateTransformMappingsTests extends AbstractWatcherIntegrationTestCase {
|
||||
|
||||
@Override
|
||||
|
@ -88,7 +90,7 @@ public class HistoryTemplateTransformMappingsTests extends AbstractWatcherIntegr
|
|||
.addAction("logger", scriptTransform("return [ 'key' : 'value2' ];"), loggingAction("indexed")))
|
||||
.get();
|
||||
assertThat(putWatchResponse.isCreated(), is(true));
|
||||
timeWarp().scheduler().trigger("_id1");
|
||||
timeWarp().trigger("_id1");
|
||||
|
||||
// adding another watch which with a transform that should conflict with the preview watch. Since the
|
||||
// mapping for the transform construct is disabled, there should be nor problems.
|
||||
|
@ -100,7 +102,7 @@ public class HistoryTemplateTransformMappingsTests extends AbstractWatcherIntegr
|
|||
.addAction("logger", scriptTransform("return [ 'key' : [ 'key1' : 'value2' ] ];"), loggingAction("indexed")))
|
||||
.get();
|
||||
assertThat(putWatchResponse.isCreated(), is(true));
|
||||
timeWarp().scheduler().trigger("_id2");
|
||||
timeWarp().trigger("_id2");
|
||||
|
||||
flush();
|
||||
refresh();
|
||||
|
|
|
@ -75,7 +75,7 @@ public class ChainIntegrationTests extends AbstractWatcherIntegrationTestCase {
|
|||
.get();
|
||||
|
||||
if (timeWarped()) {
|
||||
timeWarp().scheduler().trigger("_name");
|
||||
timeWarp().trigger("_name");
|
||||
refresh();
|
||||
} else {
|
||||
assertBusy(() -> assertWatchExecuted(), 9, TimeUnit.SECONDS);
|
||||
|
|
|
@ -12,6 +12,7 @@ import org.elasticsearch.common.settings.Settings;
|
|||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.common.xcontent.XContentType;
|
||||
import org.elasticsearch.plugins.Plugin;
|
||||
import org.elasticsearch.test.junit.annotations.TestLogging;
|
||||
import org.elasticsearch.transport.Netty4Plugin;
|
||||
import org.elasticsearch.xpack.common.http.HttpRequestTemplate;
|
||||
import org.elasticsearch.xpack.common.http.auth.basic.BasicAuth;
|
||||
|
@ -41,7 +42,9 @@ import static org.elasticsearch.xpack.watcher.trigger.TriggerBuilders.schedule;
|
|||
import static org.elasticsearch.xpack.watcher.trigger.schedule.Schedules.interval;
|
||||
import static org.hamcrest.Matchers.equalTo;
|
||||
|
||||
@TestLogging("org.elasticsearch.xpack.watcher:DEBUG,org.elasticsearch.xpack.watcher.WatcherIndexingListener:TRACE")
|
||||
public class HttpInputIntegrationTests extends AbstractWatcherIntegrationTestCase {
|
||||
|
||||
@Override
|
||||
protected Settings nodeSettings(int nodeOrdinal) {
|
||||
return Settings.builder()
|
||||
|
@ -50,6 +53,11 @@ public class HttpInputIntegrationTests extends AbstractWatcherIntegrationTestCas
|
|||
.build();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean timeWarped() {
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Collection<Class<? extends Plugin>> nodePlugins() {
|
||||
ArrayList<Class<? extends Plugin>> plugins = new ArrayList<>(super.nodePlugins());
|
||||
|
@ -74,10 +82,8 @@ public class HttpInputIntegrationTests extends AbstractWatcherIntegrationTestCas
|
|||
.addAction("_id", loggingAction("anything")))
|
||||
.get();
|
||||
|
||||
if (timeWarped()) {
|
||||
timeWarp().scheduler().trigger("_name");
|
||||
refresh();
|
||||
}
|
||||
timeWarp().trigger("_name");
|
||||
refresh();
|
||||
assertWatchWithMinimumPerformedActionsCount("_name", 1, false);
|
||||
}
|
||||
|
||||
|
@ -94,10 +100,8 @@ public class HttpInputIntegrationTests extends AbstractWatcherIntegrationTestCas
|
|||
.get();
|
||||
|
||||
assertTrue(putWatchResponse.isCreated());
|
||||
if (timeWarped()) {
|
||||
timeWarp().scheduler().trigger("_name");
|
||||
refresh();
|
||||
}
|
||||
timeWarp().trigger("_name");
|
||||
refresh();
|
||||
assertWatchWithMinimumPerformedActionsCount("_name", 1, false);
|
||||
}
|
||||
|
||||
|
@ -134,13 +138,9 @@ public class HttpInputIntegrationTests extends AbstractWatcherIntegrationTestCas
|
|||
.condition(new CompareCondition("ctx.payload.hits.max_score", CompareCondition.Op.GTE, 0L)))
|
||||
.get();
|
||||
|
||||
if (timeWarped()) {
|
||||
timeWarp().scheduler().trigger("_name1");
|
||||
timeWarp().scheduler().trigger("_name2");
|
||||
refresh();
|
||||
} else {
|
||||
Thread.sleep(10000);
|
||||
}
|
||||
timeWarp().trigger("_name1");
|
||||
timeWarp().trigger("_name2");
|
||||
refresh();
|
||||
|
||||
assertWatchWithMinimumPerformedActionsCount("_name1", 1, false);
|
||||
assertWatchWithNoActionNeeded("_name2", 1);
|
||||
|
|
|
@ -6,9 +6,8 @@
|
|||
package org.elasticsearch.xpack.watcher.security;
|
||||
|
||||
import org.elasticsearch.common.settings.SecureString;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.xpack.security.Security;
|
||||
import org.elasticsearch.xpack.watcher.WatcherState;
|
||||
import org.elasticsearch.xpack.watcher.client.WatcherClient;
|
||||
import org.elasticsearch.xpack.watcher.test.AbstractWatcherIntegrationTestCase;
|
||||
import org.elasticsearch.xpack.watcher.transport.actions.delete.DeleteWatchResponse;
|
||||
import org.elasticsearch.xpack.watcher.transport.actions.execute.ExecuteWatchResponse;
|
||||
|
@ -18,13 +17,14 @@ import org.elasticsearch.xpack.watcher.transport.actions.stats.WatcherStatsRespo
|
|||
import org.elasticsearch.xpack.watcher.trigger.TriggerEvent;
|
||||
import org.elasticsearch.xpack.watcher.trigger.schedule.IntervalSchedule;
|
||||
import org.elasticsearch.xpack.watcher.trigger.schedule.ScheduleTriggerEvent;
|
||||
import org.elasticsearch.xpack.watcher.watch.Watch;
|
||||
import org.joda.time.DateTime;
|
||||
|
||||
import java.util.Collections;
|
||||
|
||||
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
|
||||
import static org.elasticsearch.xpack.security.authc.support.UsernamePasswordToken.basicAuthHeaderValue;
|
||||
import static org.elasticsearch.xpack.security.authc.support.UsernamePasswordToken
|
||||
.BASIC_AUTH_HEADER;
|
||||
import static org.elasticsearch.xpack.security.authc.support.UsernamePasswordToken
|
||||
.basicAuthHeaderValue;
|
||||
import static org.elasticsearch.xpack.watcher.client.WatchSourceBuilders.watchBuilder;
|
||||
import static org.elasticsearch.xpack.watcher.trigger.TriggerBuilders.schedule;
|
||||
import static org.elasticsearch.xpack.watcher.trigger.schedule.Schedules.interval;
|
||||
|
@ -33,35 +33,20 @@ import static org.hamcrest.core.Is.is;
|
|||
import static org.joda.time.DateTimeZone.UTC;
|
||||
|
||||
public class BasicSecurityTests extends AbstractWatcherIntegrationTestCase {
|
||||
|
||||
@Override
|
||||
protected boolean enableSecurity() {
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Settings transportClientSettings() {
|
||||
return Settings.builder()
|
||||
.put(super.transportClientSettings())
|
||||
// Use just the transport user here, so we can test Watcher roles specifically
|
||||
.put(Security.USER_SETTING.getKey(), "transport_client:changeme")
|
||||
.build();
|
||||
}
|
||||
|
||||
public void testNoAuthorization() throws Exception {
|
||||
try {
|
||||
watcherClient().prepareWatcherStats().get();
|
||||
fail("authentication failure should have occurred");
|
||||
} catch (Exception e) {
|
||||
// transport_client is the default user
|
||||
assertThat(e.getMessage(), equalTo("action [cluster:monitor/xpack/watcher/stats] is unauthorized for user [transport_client]"));
|
||||
}
|
||||
String basicAuth = basicAuthHeaderValue("transport_client", new SecureString("changeme".toCharArray()));
|
||||
WatcherClient watcherClient = watcherClient().filterWithHeader(Collections.singletonMap(BASIC_AUTH_HEADER, basicAuth));
|
||||
Exception e = expectThrows(Exception.class, () -> watcherClient.prepareWatcherStats().get());
|
||||
assertThat(e.getMessage(), equalTo("action [cluster:monitor/xpack/watcher/stats] is unauthorized for user [transport_client]"));
|
||||
}
|
||||
|
||||
public void testWatcherMonitorRole() throws Exception {
|
||||
if (client().admin().indices().prepareExists(Watch.INDEX).get().isExists() == false) {
|
||||
assertAcked(client().admin().indices().prepareCreate(Watch.INDEX));
|
||||
}
|
||||
|
||||
// stats and get watch apis require at least monitor role:
|
||||
String token = basicAuthHeaderValue("test", new SecureString("changeme".toCharArray()));
|
||||
try {
|
||||
|
@ -84,7 +69,8 @@ public class BasicSecurityTests extends AbstractWatcherIntegrationTestCase {
|
|||
token = basicAuthHeaderValue("monitor", new SecureString("changeme".toCharArray()));
|
||||
WatcherStatsResponse statsResponse = watcherClient().filterWithHeader(Collections.singletonMap("Authorization", token))
|
||||
.prepareWatcherStats().get();
|
||||
assertThat(statsResponse.getWatcherState(), equalTo(WatcherState.STARTED));
|
||||
boolean watcherStarted = statsResponse.getNodes().stream().anyMatch(node -> node.getWatcherState() == WatcherState.STARTED);
|
||||
assertThat(watcherStarted, is(true));
|
||||
GetWatchResponse getWatchResponse = watcherClient().filterWithHeader(Collections.singletonMap("Authorization", token))
|
||||
.prepareGetWatch("_id").get();
|
||||
assertThat(getWatchResponse.isFound(), is(false));
|
||||
|
@ -152,7 +138,8 @@ public class BasicSecurityTests extends AbstractWatcherIntegrationTestCase {
|
|||
WatcherStatsResponse statsResponse = watcherClient().filterWithHeader(Collections.singletonMap("Authorization", token))
|
||||
.prepareWatcherStats()
|
||||
.get();
|
||||
assertThat(statsResponse.getWatcherState(), equalTo(WatcherState.STARTED));
|
||||
boolean watcherStarted = statsResponse.getNodes().stream().anyMatch(node -> node.getWatcherState() == WatcherState.STARTED);
|
||||
assertThat(watcherStarted, is(true));
|
||||
GetWatchResponse getWatchResponse = watcherClient().filterWithHeader(Collections.singletonMap("Authorization", token))
|
||||
.prepareGetWatch("_id")
|
||||
.get();
|
||||
|
|
|
@ -6,7 +6,6 @@
|
|||
package org.elasticsearch.xpack.watcher.test;
|
||||
|
||||
import io.netty.util.internal.SystemPropertyUtil;
|
||||
|
||||
import org.elasticsearch.ElasticsearchException;
|
||||
import org.elasticsearch.action.admin.indices.alias.Alias;
|
||||
import org.elasticsearch.action.admin.indices.create.CreateIndexResponse;
|
||||
|
@ -19,7 +18,9 @@ import org.elasticsearch.client.node.NodeClient;
|
|||
import org.elasticsearch.cluster.ClusterState;
|
||||
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
|
||||
import org.elasticsearch.cluster.routing.IndexRoutingTable;
|
||||
import org.elasticsearch.cluster.service.ClusterService;
|
||||
import org.elasticsearch.common.bytes.BytesReference;
|
||||
import org.elasticsearch.common.collect.Tuple;
|
||||
import org.elasticsearch.common.io.Streams;
|
||||
import org.elasticsearch.common.network.NetworkModule;
|
||||
import org.elasticsearch.common.settings.ClusterSettings;
|
||||
|
@ -27,6 +28,7 @@ import org.elasticsearch.common.settings.SecureString;
|
|||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.unit.TimeValue;
|
||||
import org.elasticsearch.common.util.Callback;
|
||||
import org.elasticsearch.common.util.iterable.Iterables;
|
||||
import org.elasticsearch.common.xcontent.XContentHelper;
|
||||
import org.elasticsearch.common.xcontent.XContentParser;
|
||||
import org.elasticsearch.common.xcontent.json.JsonXContent;
|
||||
|
@ -62,8 +64,6 @@ import org.elasticsearch.xpack.security.authc.support.Hasher;
|
|||
import org.elasticsearch.xpack.security.crypto.CryptoService;
|
||||
import org.elasticsearch.xpack.support.clock.ClockMock;
|
||||
import org.elasticsearch.xpack.template.TemplateUtils;
|
||||
import org.elasticsearch.xpack.watcher.WatcherLifeCycleService;
|
||||
import org.elasticsearch.xpack.watcher.WatcherService;
|
||||
import org.elasticsearch.xpack.watcher.WatcherState;
|
||||
import org.elasticsearch.xpack.watcher.client.WatcherClient;
|
||||
import org.elasticsearch.xpack.watcher.execution.ExecutionState;
|
||||
|
@ -71,9 +71,12 @@ import org.elasticsearch.xpack.watcher.execution.TriggeredWatchStore;
|
|||
import org.elasticsearch.xpack.watcher.history.HistoryStore;
|
||||
import org.elasticsearch.xpack.watcher.support.WatcherIndexTemplateRegistry;
|
||||
import org.elasticsearch.xpack.watcher.support.xcontent.XContentSource;
|
||||
import org.elasticsearch.xpack.watcher.transport.actions.stats.WatcherStatsResponse;
|
||||
import org.elasticsearch.xpack.watcher.trigger.ScheduleTriggerEngineMock;
|
||||
import org.elasticsearch.xpack.watcher.watch.Watch;
|
||||
import org.hamcrest.Matcher;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.DateTimeZone;
|
||||
import org.junit.After;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Before;
|
||||
|
@ -90,10 +93,12 @@ import java.util.Collection;
|
|||
import java.util.Collections;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Locale;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
import java.util.function.Function;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static org.elasticsearch.index.query.QueryBuilders.boolQuery;
|
||||
import static org.elasticsearch.index.query.QueryBuilders.matchQuery;
|
||||
|
@ -105,6 +110,7 @@ import static org.elasticsearch.xpack.watcher.support.WatcherIndexTemplateRegist
|
|||
import static org.elasticsearch.xpack.watcher.support.WatcherIndexTemplateRegistry.WATCHES_TEMPLATE_NAME;
|
||||
import static org.hamcrest.Matchers.emptyArray;
|
||||
import static org.hamcrest.Matchers.equalTo;
|
||||
import static org.hamcrest.Matchers.everyItem;
|
||||
import static org.hamcrest.Matchers.greaterThanOrEqualTo;
|
||||
import static org.hamcrest.Matchers.notNullValue;
|
||||
import static org.hamcrest.core.Is.is;
|
||||
|
@ -228,25 +234,18 @@ public abstract class AbstractWatcherIntegrationTestCase extends ESIntegTestCase
|
|||
return randomBoolean();
|
||||
}
|
||||
|
||||
protected boolean checkWatcherRunningOnlyOnce() {
|
||||
return true;
|
||||
}
|
||||
|
||||
@Before
|
||||
public void _setup() throws Exception {
|
||||
setupTimeWarp();
|
||||
startWatcherIfNodesExist();
|
||||
configureAliasesForWatcherIndices();
|
||||
createWatcherIndicesOrAliases();
|
||||
}
|
||||
|
||||
@After
|
||||
public void _cleanup() throws Exception {
|
||||
// Clear all internal watcher state for the next test method:
|
||||
logger.info("[{}#{}]: clearing watcher state", getTestClass().getSimpleName(), getTestName());
|
||||
if (checkWatcherRunningOnlyOnce()) {
|
||||
ensureWatcherOnlyRunningOnce();
|
||||
}
|
||||
stopWatcher(false);
|
||||
logger.info("[#{}]: clearing watcher state", getTestName());
|
||||
stopWatcher();
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
|
@ -270,7 +269,8 @@ public abstract class AbstractWatcherIntegrationTestCase extends ESIntegTestCase
|
|||
|
||||
private void setupTimeWarp() throws Exception {
|
||||
if (timeWarped()) {
|
||||
timeWarp = new TimeWarp(getInstanceFromMaster(ScheduleTriggerEngineMock.class), (ClockMock)getInstanceFromMaster(Clock.class));
|
||||
timeWarp = new TimeWarp(internalCluster().getInstances(ScheduleTriggerEngineMock.class),
|
||||
(ClockMock)getInstanceFromMaster(Clock.class));
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -284,21 +284,7 @@ public abstract class AbstractWatcherIntegrationTestCase extends ESIntegTestCase
|
|||
internalCluster().setDisruptionScheme(ice);
|
||||
ice.startDisrupting();
|
||||
}
|
||||
WatcherState state = getInstanceFromMaster(WatcherService.class).state();
|
||||
if (state == WatcherState.STOPPED) {
|
||||
logger.info("[{}#{}]: starting watcher", getTestClass().getSimpleName(), getTestName());
|
||||
startWatcher(false);
|
||||
} else if (state == WatcherState.STARTING) {
|
||||
logger.info("[{}#{}]: watcher is starting, waiting for it to get in a started state",
|
||||
getTestClass().getSimpleName(), getTestName());
|
||||
ensureWatcherStarted(false);
|
||||
} else {
|
||||
logger.info("[{}#{}]: not starting watcher, because watcher is in state [{}]",
|
||||
getTestClass().getSimpleName(), getTestName(), state);
|
||||
}
|
||||
} else {
|
||||
logger.info("[{}#{}]: not starting watcher, because test cluster has no nodes",
|
||||
getTestClass().getSimpleName(), getTestName());
|
||||
assertAcked(watcherClient().prepareWatchService().start().get());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -310,7 +296,7 @@ public abstract class AbstractWatcherIntegrationTestCase extends ESIntegTestCase
|
|||
* https://github.com/elastic/elasticsearch-migration/
|
||||
*
|
||||
*/
|
||||
private void configureAliasesForWatcherIndices() throws Exception {
|
||||
private void createWatcherIndicesOrAliases() throws Exception {
|
||||
if (internalCluster().size() > 0) {
|
||||
// alias for .watches, setting the index template to the same as well
|
||||
if (rarely()) {
|
||||
|
@ -327,7 +313,22 @@ public abstract class AbstractWatcherIntegrationTestCase extends ESIntegTestCase
|
|||
.addMapping("watch", (Map<String, Object>) allMappings.get("watch"))
|
||||
.get();
|
||||
assertAcked(response);
|
||||
ensureGreen(newIndex);
|
||||
}
|
||||
} else {
|
||||
Settings.Builder builder = Settings.builder();
|
||||
if (randomBoolean()) {
|
||||
builder.put("index.number_of_shards", scaledRandomIntBetween(1, 5));
|
||||
}
|
||||
if (randomBoolean()) {
|
||||
// maximum number of replicas
|
||||
ClusterState state = internalCluster().getDataNodeInstance(ClusterService.class).state();
|
||||
int dataNodeCount = state.nodes().getDataNodes().size();
|
||||
int replicas = scaledRandomIntBetween(0, dataNodeCount - 1);
|
||||
builder.put("index.number_of_replicas", replicas);
|
||||
}
|
||||
assertAcked(client().admin().indices().prepareCreate(Watch.INDEX).setSettings(builder));
|
||||
ensureGreen(Watch.INDEX);
|
||||
}
|
||||
|
||||
// alias for .triggered-watches, ensuring the index template is set appropriately
|
||||
|
@ -345,8 +346,18 @@ public abstract class AbstractWatcherIntegrationTestCase extends ESIntegTestCase
|
|||
.addMapping("triggered_watch", (Map<String, Object>) allMappings.get("triggered_watch"))
|
||||
.get();
|
||||
assertAcked(response);
|
||||
ensureGreen(newIndex);
|
||||
}
|
||||
} else {
|
||||
assertAcked(client().admin().indices().prepareCreate(TriggeredWatchStore.INDEX_NAME));
|
||||
ensureGreen(TriggeredWatchStore.INDEX_NAME);
|
||||
}
|
||||
|
||||
String historyIndex = HistoryStore.getHistoryIndexNameForTime(DateTime.now(DateTimeZone.UTC));
|
||||
assertAcked(client().admin().indices().prepareCreate(historyIndex));
|
||||
ensureGreen(historyIndex);
|
||||
|
||||
ensureWatcherStarted();
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -551,19 +562,7 @@ public abstract class AbstractWatcherIntegrationTestCase extends ESIntegTestCase
|
|||
});
|
||||
}
|
||||
|
||||
protected void ensureWatcherStarted() throws Exception {
|
||||
ensureWatcherStarted(true);
|
||||
}
|
||||
|
||||
protected void ensureWatcherStarted(final boolean useClient) throws Exception {
|
||||
assertBusy(() -> {
|
||||
if (useClient) {
|
||||
assertThat(watcherClient().prepareWatcherStats().get().getWatcherState(), is(WatcherState.STARTED));
|
||||
} else {
|
||||
assertThat(getInstanceFromMaster(WatcherService.class).state(), is(WatcherState.STARTED));
|
||||
}
|
||||
});
|
||||
|
||||
private void ensureWatcherTemplatesAdded() throws Exception {
|
||||
// Verify that the index templates exist:
|
||||
assertBusy(() -> {
|
||||
GetIndexTemplatesResponse response = client().admin().indices().prepareGetTemplates(HISTORY_TEMPLATE_NAME).get();
|
||||
|
@ -575,6 +574,22 @@ public abstract class AbstractWatcherIntegrationTestCase extends ESIntegTestCase
|
|||
});
|
||||
}
|
||||
|
||||
protected void ensureWatcherStarted() throws Exception {
|
||||
ensureWatcherTemplatesAdded();
|
||||
assertBusy(() -> {
|
||||
WatcherStatsResponse watcherStatsResponse = watcherClient().prepareWatcherStats().get();
|
||||
assertThat(watcherStatsResponse.hasFailures(), is(false));
|
||||
List<Tuple<String, WatcherState>> currentStatesFromStatsRequest = watcherStatsResponse.getNodes().stream()
|
||||
.map(response -> Tuple.tuple(response.getNode().getName(), response.getWatcherState()))
|
||||
.collect(Collectors.toList());
|
||||
List<WatcherState> states = currentStatesFromStatsRequest.stream().map(Tuple::v2).collect(Collectors.toList());
|
||||
|
||||
String message = String.format(Locale.ROOT, "Expected watcher to be started, but state was %s", currentStatesFromStatsRequest);
|
||||
assertThat(message, states, everyItem(is(WatcherState.STARTED)));
|
||||
});
|
||||
|
||||
}
|
||||
|
||||
protected void ensureLicenseEnabled() throws Exception {
|
||||
assertBusy(() -> {
|
||||
for (XPackLicenseState licenseState : internalCluster().getInstances(XPackLicenseState.class)) {
|
||||
|
@ -583,69 +598,28 @@ public abstract class AbstractWatcherIntegrationTestCase extends ESIntegTestCase
|
|||
});
|
||||
}
|
||||
|
||||
private void progressClocksAboveMaster(InternalTestCluster cluster) {
|
||||
long minClock = Long.MAX_VALUE;
|
||||
long maxClock = Long.MIN_VALUE;
|
||||
for (Clock clock: cluster.getInstances(Clock.class)) {
|
||||
final long millis = clock.millis();
|
||||
minClock = Math.min(millis, minClock);
|
||||
maxClock = Math.max(millis, maxClock);
|
||||
}
|
||||
// now move all the clocks ahead to make sure they are beyond the highest clock
|
||||
final TimeValue delta = TimeValue.timeValueMillis(maxClock - minClock);
|
||||
for (Clock clock: cluster.getInstances(Clock.class)) {
|
||||
((ClockMock)clock).fastForward(delta);
|
||||
}
|
||||
}
|
||||
|
||||
protected void ensureWatcherStopped() throws Exception {
|
||||
ensureWatcherStopped(true);
|
||||
}
|
||||
|
||||
protected void ensureWatcherStopped(final boolean useClient) throws Exception {
|
||||
assertBusy(() -> {
|
||||
if (useClient) {
|
||||
assertThat(watcherClient().prepareWatcherStats().get().getWatcherState(), is(WatcherState.STOPPED));
|
||||
} else {
|
||||
assertThat(getInstanceFromMaster(WatcherService.class).state(), is(WatcherState.STOPPED));
|
||||
}
|
||||
WatcherStatsResponse watcherStatsResponse = watcherClient().prepareWatcherStats().get();
|
||||
assertThat(watcherStatsResponse.hasFailures(), is(false));
|
||||
List<Tuple<String, WatcherState>> currentStatesFromStatsRequest = watcherStatsResponse.getNodes().stream()
|
||||
.map(response -> Tuple.tuple(response.getNode().getName(), response.getWatcherState()))
|
||||
.collect(Collectors.toList());
|
||||
List<WatcherState> states = currentStatesFromStatsRequest.stream().map(Tuple::v2).collect(Collectors.toList());
|
||||
|
||||
String message = String.format(Locale.ROOT, "Expected watcher to be stopped, but state was %s", currentStatesFromStatsRequest);
|
||||
assertThat(message, states, everyItem(is(WatcherState.STOPPED)));
|
||||
});
|
||||
}
|
||||
|
||||
protected void startWatcher() throws Exception {
|
||||
startWatcher(true);
|
||||
watcherClient().prepareWatchService().start().get();
|
||||
ensureWatcherStarted();
|
||||
}
|
||||
|
||||
protected void stopWatcher() throws Exception {
|
||||
stopWatcher(true);
|
||||
}
|
||||
|
||||
protected void startWatcher(boolean useClient) throws Exception {
|
||||
if (useClient) {
|
||||
watcherClient().prepareWatchService().start().get();
|
||||
} else {
|
||||
getInstanceFromMaster(WatcherLifeCycleService.class).start();
|
||||
}
|
||||
ensureWatcherStarted(useClient);
|
||||
}
|
||||
|
||||
protected void stopWatcher(boolean useClient) throws Exception {
|
||||
if (useClient) {
|
||||
watcherClient().prepareWatchService().stop().get();
|
||||
} else {
|
||||
getInstanceFromMaster(WatcherLifeCycleService.class).stop();
|
||||
}
|
||||
ensureWatcherStopped(useClient);
|
||||
}
|
||||
|
||||
protected void ensureWatcherOnlyRunningOnce() {
|
||||
int running = 0;
|
||||
for (WatcherService watcherService : internalCluster().getInstances(WatcherService.class)) {
|
||||
if (watcherService.state() == WatcherState.STARTED) {
|
||||
running++;
|
||||
}
|
||||
}
|
||||
assertThat("watcher should only run on the elected master node, but it is running on [" + running + "] nodes", running, equalTo(1));
|
||||
watcherClient().prepareWatchService().stop().get();
|
||||
ensureWatcherStopped();
|
||||
}
|
||||
|
||||
public static class NoopEmailService extends EmailService {
|
||||
|
@ -663,21 +637,25 @@ public abstract class AbstractWatcherIntegrationTestCase extends ESIntegTestCase
|
|||
|
||||
protected static class TimeWarp {
|
||||
|
||||
protected final ScheduleTriggerEngineMock scheduler;
|
||||
protected final Iterable<ScheduleTriggerEngineMock> schedulers;
|
||||
protected final ClockMock clock;
|
||||
|
||||
public TimeWarp(ScheduleTriggerEngineMock scheduler, ClockMock clock) {
|
||||
this.scheduler = scheduler;
|
||||
public TimeWarp(Iterable<ScheduleTriggerEngineMock> schedulers, ClockMock clock) {
|
||||
this.schedulers = schedulers;
|
||||
this.clock = clock;
|
||||
}
|
||||
|
||||
public ScheduleTriggerEngineMock scheduler() {
|
||||
return scheduler;
|
||||
public void trigger(String jobName) {
|
||||
schedulers.forEach(scheduler -> scheduler.trigger(jobName));
|
||||
}
|
||||
|
||||
public ClockMock clock() {
|
||||
return clock;
|
||||
}
|
||||
|
||||
public void trigger(String id, int times, TimeValue timeValue) {
|
||||
schedulers.forEach(scheduler -> scheduler.trigger(id, times, timeValue));
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
|
|
|
@ -5,6 +5,7 @@
|
|||
*/
|
||||
package org.elasticsearch.xpack.watcher.test;
|
||||
|
||||
import org.apache.logging.log4j.Logger;
|
||||
import org.elasticsearch.common.logging.Loggers;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
|
@ -25,10 +26,11 @@ import java.util.stream.Stream;
|
|||
|
||||
public class TimeWarpedWatcher extends Watcher {
|
||||
|
||||
private static final Logger logger = Loggers.getLogger(TimeWarpedWatcher.class);
|
||||
|
||||
public TimeWarpedWatcher(Settings settings) {
|
||||
super(settings);
|
||||
Loggers.getLogger(TimeWarpedWatcher.class, settings).info("using time warped watchers plugin");
|
||||
|
||||
logger.info("using time warped watchers plugin");
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -147,7 +147,8 @@ public class WatcherScheduleEngineBenchmark {
|
|||
|
||||
Clock clock = node.injector().getInstance(Clock.class);
|
||||
WatcherClient watcherClient = node.injector().getInstance(WatcherClient.class);
|
||||
while (watcherClient.prepareWatcherStats().get().getWatcherState() != WatcherState.STARTED) {
|
||||
while (!watcherClient.prepareWatcherStats().get().getNodes().stream()
|
||||
.allMatch(r -> r.getWatcherState() == WatcherState.STARTED)) {
|
||||
Thread.sleep(100);
|
||||
}
|
||||
long actualLoadedWatches = watcherClient.prepareWatcherStats().get().getWatchesCount();
|
||||
|
|
|
@ -8,7 +8,6 @@ package org.elasticsearch.xpack.watcher.test.integration;
|
|||
import org.elasticsearch.ElasticsearchParseException;
|
||||
import org.elasticsearch.action.search.SearchResponse;
|
||||
import org.elasticsearch.action.search.SearchType;
|
||||
import org.elasticsearch.common.unit.TimeValue;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.common.xcontent.XContentType;
|
||||
import org.elasticsearch.script.Script;
|
||||
|
@ -60,6 +59,8 @@ import static org.hamcrest.Matchers.equalTo;
|
|||
import static org.hamcrest.Matchers.is;
|
||||
import static org.hamcrest.Matchers.notNullValue;
|
||||
|
||||
@TestLogging("org.elasticsearch.xpack.watcher:DEBUG," +
|
||||
"org.elasticsearch.xpack.watcher.WatcherIndexingListener:TRACE")
|
||||
public class BasicWatcherTests extends AbstractWatcherIntegrationTestCase {
|
||||
|
||||
@Override
|
||||
|
@ -76,18 +77,19 @@ public class BasicWatcherTests extends AbstractWatcherIntegrationTestCase {
|
|||
WatcherClient watcherClient = watcherClient();
|
||||
createIndex("idx");
|
||||
// Have a sample document in the index, the watch is going to evaluate
|
||||
client().prepareIndex("idx", "type").setSource("field", "value").get();
|
||||
client().prepareIndex("idx", "type").setSource("field", "foo").get();
|
||||
refresh();
|
||||
WatcherSearchTemplateRequest request = templateRequest(searchSource().query(termQuery("field", "value")), "idx");
|
||||
WatcherSearchTemplateRequest request = templateRequest(searchSource().query(termQuery("field", "foo")), "idx");
|
||||
watcherClient.preparePutWatch("_name")
|
||||
.setSource(watchBuilder()
|
||||
.trigger(schedule(interval(5, IntervalSchedule.Interval.Unit.SECONDS)))
|
||||
.input(searchInput(request))
|
||||
.condition(new CompareCondition("ctx.payload.hits.total", CompareCondition.Op.EQ, 1L))
|
||||
.addAction("_logger", loggingAction("_logging")
|
||||
.setCategory("_category")))
|
||||
.setCategory("_category")))
|
||||
.get();
|
||||
timeWarp().scheduler().trigger("_name");
|
||||
ensureWatcherStarted();
|
||||
timeWarp().trigger("_name");
|
||||
assertWatchWithMinimumPerformedActionsCount("_name", 1);
|
||||
|
||||
GetWatchResponse getWatchResponse = watcherClient().prepareGetWatch().setId("_name").get();
|
||||
|
@ -104,7 +106,7 @@ public class BasicWatcherTests extends AbstractWatcherIntegrationTestCase {
|
|||
.input(searchInput(searchRequest))
|
||||
.condition(new CompareCondition("ctx.payload.hits.total", CompareCondition.Op.EQ, 1L)))
|
||||
.get();
|
||||
timeWarp().scheduler().trigger("_name");
|
||||
timeWarp().trigger("_name");
|
||||
// The watch's condition won't meet because there is no data that matches with the query
|
||||
assertWatchWithNoActionNeeded("_name", 1);
|
||||
|
||||
|
@ -114,7 +116,7 @@ public class BasicWatcherTests extends AbstractWatcherIntegrationTestCase {
|
|||
|
||||
if (timeWarped()) {
|
||||
timeWarp().clock().fastForwardSeconds(5);
|
||||
timeWarp().scheduler().trigger("_name");
|
||||
timeWarp().trigger("_name");
|
||||
refresh();
|
||||
}
|
||||
|
||||
|
@ -178,7 +180,9 @@ public class BasicWatcherTests extends AbstractWatcherIntegrationTestCase {
|
|||
}
|
||||
}
|
||||
|
||||
@TestLogging("org.elasticsearch.xpack.watcher:DEBUG")
|
||||
public void testModifyWatches() throws Exception {
|
||||
createIndex("idx");
|
||||
WatcherSearchTemplateRequest searchRequest = templateRequest(searchSource().query(matchAllQuery()), "idx");
|
||||
|
||||
WatchSourceBuilder source = watchBuilder()
|
||||
|
@ -191,7 +195,7 @@ public class BasicWatcherTests extends AbstractWatcherIntegrationTestCase {
|
|||
.get();
|
||||
|
||||
timeWarp().clock().fastForwardSeconds(5);
|
||||
timeWarp().scheduler().trigger("_name");
|
||||
timeWarp().trigger("_name");
|
||||
assertWatchWithMinimumPerformedActionsCount("_name", 0, false);
|
||||
|
||||
watcherClient().preparePutWatch("_name")
|
||||
|
@ -199,7 +203,7 @@ public class BasicWatcherTests extends AbstractWatcherIntegrationTestCase {
|
|||
.get();
|
||||
|
||||
timeWarp().clock().fastForwardSeconds(5);
|
||||
timeWarp().scheduler().trigger("_name");
|
||||
timeWarp().trigger("_name");
|
||||
refresh();
|
||||
assertWatchWithMinimumPerformedActionsCount("_name", 1, false);
|
||||
|
||||
|
@ -210,50 +214,14 @@ public class BasicWatcherTests extends AbstractWatcherIntegrationTestCase {
|
|||
.get();
|
||||
|
||||
timeWarp().clock().fastForwardSeconds(5);
|
||||
timeWarp().scheduler().trigger("_name");
|
||||
long count = findNumberOfPerformedActions("_name");
|
||||
timeWarp().trigger("_name");
|
||||
long count = findNumberOfPerformedActions("_name");
|
||||
|
||||
timeWarp().clock().fastForwardSeconds(5);
|
||||
timeWarp().scheduler().trigger("_name");
|
||||
timeWarp().trigger("_name");
|
||||
assertThat(count, equalTo(findNumberOfPerformedActions("_name")));
|
||||
}
|
||||
|
||||
public void testModifyWatchWithSameUnit() throws Exception {
|
||||
if (timeWarped()) {
|
||||
logger.info("Skipping testModifyWatches_ because timewarp is enabled");
|
||||
return;
|
||||
}
|
||||
|
||||
WatchSourceBuilder source = watchBuilder()
|
||||
.trigger(schedule(interval("1s")))
|
||||
.input(simpleInput("key", "value"))
|
||||
.defaultThrottlePeriod(TimeValue.timeValueSeconds(0))
|
||||
.addAction("_id", loggingAction("_logging"));
|
||||
watcherClient().preparePutWatch("_name")
|
||||
.setSource(source)
|
||||
.get();
|
||||
|
||||
Thread.sleep(5000);
|
||||
assertWatchWithMinimumPerformedActionsCount("_name", 5, false);
|
||||
|
||||
source = watchBuilder()
|
||||
.trigger(schedule(interval("100s")))
|
||||
.defaultThrottlePeriod(TimeValue.timeValueSeconds(0))
|
||||
.input(simpleInput("key", "value"))
|
||||
.addAction("_id", loggingAction("_logging"));
|
||||
watcherClient().preparePutWatch("_name")
|
||||
.setSource(source)
|
||||
.get();
|
||||
|
||||
// Wait one second to be sure that the scheduler engine has executed any previous job instance of the watch
|
||||
Thread.sleep(1000);
|
||||
long before = historyRecordsCount("_name");
|
||||
Thread.sleep(5000);
|
||||
assertThat("Watch has been updated to 100s interval, so no new records should have been added.", historyRecordsCount("_name"),
|
||||
equalTo(before));
|
||||
}
|
||||
|
||||
@TestLogging("org.elasticsearch.xpack.watcher.trigger:DEBUG")
|
||||
public void testConditionSearchWithSource() throws Exception {
|
||||
SearchSourceBuilder searchSourceBuilder = searchSource().query(matchQuery("level", "a"));
|
||||
testConditionSearch(templateRequest(searchSourceBuilder, "events"));
|
||||
|
@ -277,9 +245,9 @@ public class BasicWatcherTests extends AbstractWatcherIntegrationTestCase {
|
|||
WatcherClient watcherClient = watcherClient();
|
||||
createIndex("idx");
|
||||
// Have a sample document in the index, the watch is going to evaluate
|
||||
client().prepareIndex("idx", "type").setSource("field", "value").get();
|
||||
client().prepareIndex("idx", "type").setSource(jsonBuilder().startObject().field("field", "foovalue").endObject()).get();
|
||||
refresh();
|
||||
WatcherSearchTemplateRequest request = templateRequest(searchSource().query(termQuery("field", "value")), "idx");
|
||||
WatcherSearchTemplateRequest request = templateRequest(searchSource().query(termQuery("field", "foovalue")), "idx");
|
||||
watcherClient.preparePutWatch("_name1")
|
||||
.setSource(watchBuilder()
|
||||
.trigger(schedule(interval(5, IntervalSchedule.Interval.Unit.SECONDS)))
|
||||
|
@ -294,9 +262,9 @@ public class BasicWatcherTests extends AbstractWatcherIntegrationTestCase {
|
|||
.condition(new CompareCondition("ctx.payload.hits.max_score", CompareCondition.Op.GTE, 0L)))
|
||||
.get();
|
||||
|
||||
timeWarp().scheduler().trigger("_name1");
|
||||
timeWarp().scheduler().trigger("_name2");
|
||||
timeWarp().trigger("_name1");
|
||||
assertWatchWithMinimumPerformedActionsCount("_name1", 1);
|
||||
timeWarp().trigger("_name2");
|
||||
assertWatchWithNoActionNeeded("_name2", 1);
|
||||
|
||||
// Check that the input result payload has been filtered
|
||||
|
@ -352,10 +320,10 @@ public class BasicWatcherTests extends AbstractWatcherIntegrationTestCase {
|
|||
watcherClient().preparePutWatch("_name")
|
||||
.setSource(watchBuilder()
|
||||
.trigger(schedule(weekly().time(WeekTimes.builder().atRoundHour(-10).build()).build()))
|
||||
.input(simpleInput("key", "value"))
|
||||
.condition(AlwaysCondition.INSTANCE)
|
||||
.addAction("_logger", loggingAction("executed!")))
|
||||
.get();
|
||||
.input(simpleInput("key", "value"))
|
||||
.condition(AlwaysCondition.INSTANCE)
|
||||
.addAction("_logger", loggingAction("executed!")))
|
||||
.get();
|
||||
fail("put watch should have failed");
|
||||
} catch (IllegalArgumentException e) {
|
||||
assertThat(e.getMessage(),
|
||||
|
@ -402,7 +370,7 @@ public class BasicWatcherTests extends AbstractWatcherIntegrationTestCase {
|
|||
|
||||
refresh();
|
||||
timeWarp().clock().fastForwardSeconds(1);
|
||||
timeWarp().scheduler().trigger(watchName);
|
||||
timeWarp().trigger(watchName);
|
||||
assertWatchWithNoActionNeeded(watchName, 1);
|
||||
|
||||
client().prepareIndex("events", "event")
|
||||
|
@ -410,7 +378,7 @@ public class BasicWatcherTests extends AbstractWatcherIntegrationTestCase {
|
|||
.get();
|
||||
refresh();
|
||||
timeWarp().clock().fastForwardSeconds(1);
|
||||
timeWarp().scheduler().trigger(watchName);
|
||||
timeWarp().trigger(watchName);
|
||||
assertWatchWithNoActionNeeded(watchName, 2);
|
||||
|
||||
client().prepareIndex("events", "event")
|
||||
|
@ -418,7 +386,7 @@ public class BasicWatcherTests extends AbstractWatcherIntegrationTestCase {
|
|||
.get();
|
||||
refresh();
|
||||
timeWarp().clock().fastForwardSeconds(1);
|
||||
timeWarp().scheduler().trigger(watchName);
|
||||
timeWarp().trigger(watchName);
|
||||
assertWatchWithMinimumPerformedActionsCount(watchName, 1);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -5,12 +5,14 @@
|
|||
*/
|
||||
package org.elasticsearch.xpack.watcher.test.integration;
|
||||
|
||||
import org.elasticsearch.action.bulk.BulkRequestBuilder;
|
||||
import org.elasticsearch.action.search.SearchResponse;
|
||||
import org.elasticsearch.action.support.ActiveShardCount;
|
||||
import org.elasticsearch.action.support.WriteRequest;
|
||||
import org.elasticsearch.cluster.service.ClusterService;
|
||||
import org.elasticsearch.common.unit.TimeValue;
|
||||
import org.elasticsearch.common.xcontent.XContentType;
|
||||
import org.elasticsearch.xpack.watcher.WatcherState;
|
||||
import org.elasticsearch.test.junit.annotations.TestLogging;
|
||||
import org.elasticsearch.xpack.watcher.condition.AlwaysCondition;
|
||||
import org.elasticsearch.xpack.watcher.condition.CompareCondition;
|
||||
import org.elasticsearch.xpack.watcher.condition.Condition;
|
||||
|
@ -28,8 +30,10 @@ import org.elasticsearch.xpack.watcher.watch.Watch;
|
|||
import org.hamcrest.Matchers;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.DateTimeZone;
|
||||
import org.junit.Before;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import static org.elasticsearch.action.support.WriteRequest.RefreshPolicy.IMMEDIATE;
|
||||
|
@ -38,6 +42,7 @@ import static org.elasticsearch.index.query.QueryBuilders.termQuery;
|
|||
import static org.elasticsearch.search.builder.SearchSourceBuilder.searchSource;
|
||||
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
|
||||
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount;
|
||||
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoFailures;
|
||||
import static org.elasticsearch.xpack.watcher.actions.ActionBuilders.indexAction;
|
||||
import static org.elasticsearch.xpack.watcher.client.WatchSourceBuilders.watchBuilder;
|
||||
import static org.elasticsearch.xpack.watcher.input.InputBuilders.searchInput;
|
||||
|
@ -49,6 +54,8 @@ import static org.hamcrest.Matchers.is;
|
|||
import static org.hamcrest.core.IsEqual.equalTo;
|
||||
import static org.joda.time.DateTimeZone.UTC;
|
||||
|
||||
@TestLogging("org.elasticsearch.xpack.watcher:DEBUG," +
|
||||
"org.elasticsearch.xpack.watcher.execution:TRACE")
|
||||
public class BootStrapTests extends AbstractWatcherIntegrationTestCase {
|
||||
|
||||
@Override
|
||||
|
@ -56,13 +63,23 @@ public class BootStrapTests extends AbstractWatcherIntegrationTestCase {
|
|||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean enableSecurity() {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Before
|
||||
public void deleteAllWatchHistoryIndices() {
|
||||
assertAcked(client().admin().indices().prepareDelete(HistoryStore.INDEX_PREFIX + "*"));
|
||||
}
|
||||
|
||||
public void testLoadMalformedWatchRecord() throws Exception {
|
||||
client().prepareIndex(Watch.INDEX, Watch.DOC_TYPE, "_id")
|
||||
.setSource(jsonBuilder().startObject()
|
||||
.startObject(Watch.Field.TRIGGER.getPreferredName())
|
||||
.startObject("schedule")
|
||||
.field("cron", "0/5 * * * * ? 2050")
|
||||
.endObject()
|
||||
.startObject("schedule")
|
||||
.field("cron", "0/5 * * * * ? 2050")
|
||||
.endObject()
|
||||
.endObject()
|
||||
.startObject(Watch.Field.ACTIONS.getPreferredName())
|
||||
.endObject()
|
||||
|
@ -114,80 +131,56 @@ public class BootStrapTests extends AbstractWatcherIntegrationTestCase {
|
|||
client().prepareIndex(index, HistoryStore.DOC_TYPE, wid.value())
|
||||
.setSource(jsonBuilder().startObject()
|
||||
.startObject(WatchRecord.Field.TRIGGER_EVENT.getPreferredName())
|
||||
.startObject("unknown").endObject()
|
||||
.startObject("unknown").endObject()
|
||||
.endObject()
|
||||
.startObject(Watch.Field.CONDITION.getPreferredName())
|
||||
.field(condition.type(), condition)
|
||||
.field(condition.type(), condition)
|
||||
.endObject()
|
||||
.startObject(Watch.Field.INPUT.getPreferredName())
|
||||
.startObject("none").endObject()
|
||||
.startObject("none").endObject()
|
||||
.endObject()
|
||||
.endObject())
|
||||
.setWaitForActiveShards(ActiveShardCount.ALL)
|
||||
.setRefreshPolicy(IMMEDIATE)
|
||||
.get();
|
||||
|
||||
ensureWatcherStarted();
|
||||
stopWatcher();
|
||||
startWatcher();
|
||||
|
||||
WatcherStatsResponse response = watcherClient().prepareWatcherStats().get();
|
||||
assertThat(response.getWatcherState(), equalTo(WatcherState.STARTED));
|
||||
assertThat(response.getWatchesCount(), equalTo(1L));
|
||||
}
|
||||
|
||||
public void testDeletedWhileQueued() throws Exception {
|
||||
if (client().admin().indices().prepareExists(Watch.INDEX).get().isExists() == false) {
|
||||
// we rarely create an .watches alias in the base class
|
||||
assertAcked(client().admin().indices().prepareCreate(Watch.INDEX));
|
||||
}
|
||||
DateTime now = DateTime.now(UTC);
|
||||
Wid wid = new Wid("_id", now);
|
||||
ScheduleTriggerEvent event = new ScheduleTriggerEvent("_id", now, now);
|
||||
|
||||
client().prepareIndex(TriggeredWatchStore.INDEX_NAME, TriggeredWatchStore.DOC_TYPE, wid.value())
|
||||
.setSource(jsonBuilder().startObject()
|
||||
.startObject(WatchRecord.Field.TRIGGER_EVENT.getPreferredName())
|
||||
.field(event.type(), event)
|
||||
.endObject()
|
||||
.endObject())
|
||||
.setWaitForActiveShards(ActiveShardCount.ALL)
|
||||
.setRefreshPolicy(IMMEDIATE)
|
||||
.get();
|
||||
|
||||
stopWatcher();
|
||||
startWatcher();
|
||||
|
||||
refresh();
|
||||
SearchResponse searchResponse = client().prepareSearch(HistoryStore.INDEX_PREFIX_WITH_TEMPLATE + "*").get();
|
||||
assertHitCount(searchResponse, 1);
|
||||
assertThat(searchResponse.getHits().getAt(0).getId(), Matchers.equalTo(wid.value()));
|
||||
assertThat(searchResponse.getHits().getAt(0).getSourceAsMap().get(WatchRecord.Field.STATE.getPreferredName()).toString(),
|
||||
equalTo(ExecutionState.NOT_EXECUTED_WATCH_MISSING.toString()));
|
||||
}
|
||||
|
||||
public void testLoadExistingWatchesUponStartup() throws Exception {
|
||||
ensureWatcherStarted();
|
||||
stopWatcher();
|
||||
|
||||
int numWatches = scaledRandomIntBetween(16, 128);
|
||||
WatcherSearchTemplateRequest request =
|
||||
templateRequest(searchSource().query(termQuery("field", "value")), "my-index");
|
||||
|
||||
BulkRequestBuilder bulkRequestBuilder = client().prepareBulk();
|
||||
for (int i = 0; i < numWatches; i++) {
|
||||
client().prepareIndex(Watch.INDEX, Watch.DOC_TYPE, "_id" + i)
|
||||
.setSource(watchBuilder()
|
||||
bulkRequestBuilder.add(
|
||||
client().prepareIndex(Watch.INDEX, Watch.DOC_TYPE, "_id" + i)
|
||||
.setSource(watchBuilder()
|
||||
.trigger(schedule(cron("0 0/5 * * * ? 2050")))
|
||||
.input(searchInput(request))
|
||||
.condition(new CompareCondition("ctx.payload.hits.total", CompareCondition.Op.EQ, 1L))
|
||||
.buildAsBytes(XContentType.JSON), XContentType.JSON
|
||||
)
|
||||
.setWaitForActiveShards(ActiveShardCount.ALL)
|
||||
.get();
|
||||
)
|
||||
.setWaitForActiveShards(ActiveShardCount.ALL));
|
||||
}
|
||||
bulkRequestBuilder.setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE).get();
|
||||
assertHitCount(client().prepareSearch(Watch.INDEX).setSize(0).get(), numWatches);
|
||||
|
||||
refresh();
|
||||
stopWatcher();
|
||||
startWatcher();
|
||||
|
||||
WatcherStatsResponse response = watcherClient().prepareWatcherStats().get();
|
||||
assertThat(response.getWatcherState(), equalTo(WatcherState.STARTED));
|
||||
assertThat(response.getWatchesCount(), equalTo((long) numWatches));
|
||||
assertBusy(() -> {
|
||||
WatcherStatsResponse response = watcherClient().prepareWatcherStats().get();
|
||||
assertThat(response.getWatchesCount(), equalTo((long) numWatches));
|
||||
});
|
||||
}
|
||||
|
||||
public void testMixedTriggeredWatchLoading() throws Exception {
|
||||
|
@ -197,7 +190,6 @@ public class BootStrapTests extends AbstractWatcherIntegrationTestCase {
|
|||
.setSource("field", "value").get();
|
||||
|
||||
WatcherStatsResponse response = watcherClient().prepareWatcherStats().get();
|
||||
assertThat(response.getWatcherState(), equalTo(WatcherState.STARTED));
|
||||
assertThat(response.getWatchesCount(), equalTo(0L));
|
||||
|
||||
WatcherSearchTemplateRequest request = templateRequest(searchSource().query(termQuery("field", "value")), "my-index");
|
||||
|
@ -206,31 +198,35 @@ public class BootStrapTests extends AbstractWatcherIntegrationTestCase {
|
|||
for (int i = 0; i < numWatches; i++) {
|
||||
String watchId = "_id" + i;
|
||||
watcherClient().preparePutWatch(watchId).setSource(watchBuilder()
|
||||
.trigger(schedule(cron("0/5 * * * * ? 2050")))
|
||||
.input(searchInput(request))
|
||||
.condition(AlwaysCondition.INSTANCE)
|
||||
.addAction("_id", indexAction("output", "test"))
|
||||
.defaultThrottlePeriod(TimeValue.timeValueMillis(0))
|
||||
.trigger(schedule(cron("0/5 * * * * ? 2050")))
|
||||
.input(searchInput(request))
|
||||
.condition(AlwaysCondition.INSTANCE)
|
||||
.addAction("_id", indexAction("output", "test"))
|
||||
.defaultThrottlePeriod(TimeValue.timeValueMillis(0))
|
||||
).get();
|
||||
}
|
||||
|
||||
ensureWatcherStarted();
|
||||
stopWatcher();
|
||||
|
||||
DateTime now = DateTime.now(UTC);
|
||||
final int numRecords = scaledRandomIntBetween(numWatches, 128);
|
||||
BulkRequestBuilder bulkRequestBuilder = client().prepareBulk();
|
||||
for (int i = 0; i < numRecords; i++) {
|
||||
String watchId = "_id" + (i % numWatches);
|
||||
now = now.plusMinutes(1);
|
||||
ScheduleTriggerEvent event = new ScheduleTriggerEvent(watchId, now, now);
|
||||
Wid wid = new Wid(watchId, now);
|
||||
TriggeredWatch triggeredWatch = new TriggeredWatch(wid, event);
|
||||
client().prepareIndex(TriggeredWatchStore.INDEX_NAME, TriggeredWatchStore.DOC_TYPE, triggeredWatch.id().value())
|
||||
.setSource(jsonBuilder().value(triggeredWatch))
|
||||
.setWaitForActiveShards(ActiveShardCount.ALL)
|
||||
.get();
|
||||
bulkRequestBuilder.add(
|
||||
client().prepareIndex(TriggeredWatchStore.INDEX_NAME, TriggeredWatchStore.DOC_TYPE, triggeredWatch.id().value())
|
||||
.setSource(jsonBuilder().value(triggeredWatch))
|
||||
.request());
|
||||
}
|
||||
bulkRequestBuilder.setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE).get();
|
||||
|
||||
stopWatcher();
|
||||
logger.info("Added [{}] triggered watches for [{}] different watches, starting watcher again", numRecords, numWatches);
|
||||
startWatcher();
|
||||
|
||||
assertSingleExecutionAndCompleteWatchHistory(numWatches, numRecords);
|
||||
}
|
||||
|
||||
|
@ -241,47 +237,52 @@ public class BootStrapTests extends AbstractWatcherIntegrationTestCase {
|
|||
.setSource("field", "value").get();
|
||||
|
||||
WatcherStatsResponse response = watcherClient().prepareWatcherStats().get();
|
||||
assertThat(response.getWatcherState(), equalTo(WatcherState.STARTED));
|
||||
assertThat(response.getWatchesCount(), equalTo(0L));
|
||||
|
||||
String watchId = "_id";
|
||||
WatcherSearchTemplateRequest request = templateRequest(searchSource().query(termQuery("field", "value")), "my-index");
|
||||
watcherClient().preparePutWatch(watchId).setSource(watchBuilder()
|
||||
.trigger(schedule(cron("0/5 * * * * ? 2050")))
|
||||
.input(searchInput(request))
|
||||
.condition(AlwaysCondition.INSTANCE)
|
||||
.addAction("_id", indexAction("output", "test"))
|
||||
.defaultThrottlePeriod(TimeValue.timeValueMillis(0))
|
||||
.trigger(schedule(cron("0/5 * * * * ? 2050")))
|
||||
.input(searchInput(request))
|
||||
.condition(AlwaysCondition.INSTANCE)
|
||||
.addAction("_id", indexAction("output", "test"))
|
||||
.defaultThrottlePeriod(TimeValue.timeValueMillis(0))
|
||||
).get();
|
||||
|
||||
ensureWatcherStarted();
|
||||
stopWatcher();
|
||||
|
||||
DateTime now = DateTime.now(UTC);
|
||||
final int numRecords = scaledRandomIntBetween(2, 12);
|
||||
// final int numRecords = scaledRandomIntBetween(2, 12);
|
||||
final int numRecords = 10;
|
||||
BulkRequestBuilder bulkRequestBuilder = client().prepareBulk();
|
||||
for (int i = 0; i < numRecords; i++) {
|
||||
now = now.plusMinutes(1);
|
||||
ScheduleTriggerEvent event = new ScheduleTriggerEvent(watchId, now, now);
|
||||
Wid wid = new Wid(watchId, now);
|
||||
TriggeredWatch triggeredWatch = new TriggeredWatch(wid, event);
|
||||
client().prepareIndex(TriggeredWatchStore.INDEX_NAME, TriggeredWatchStore.DOC_TYPE, triggeredWatch.id().value())
|
||||
bulkRequestBuilder.add(client().prepareIndex(TriggeredWatchStore.INDEX_NAME, TriggeredWatchStore.DOC_TYPE, triggeredWatch.id
|
||||
().value())
|
||||
.setSource(jsonBuilder().value(triggeredWatch))
|
||||
.setWaitForActiveShards(ActiveShardCount.ALL)
|
||||
.get();
|
||||
);
|
||||
}
|
||||
bulkRequestBuilder.setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE).get();
|
||||
|
||||
stopWatcher();
|
||||
startWatcher();
|
||||
|
||||
assertSingleExecutionAndCompleteWatchHistory(1, numRecords);
|
||||
}
|
||||
|
||||
private void assertSingleExecutionAndCompleteWatchHistory(long numberOfWatches, int expectedWatchHistoryCount) throws Exception {
|
||||
private void assertSingleExecutionAndCompleteWatchHistory(final long numberOfWatches,
|
||||
final int expectedWatchHistoryCount) throws Exception {
|
||||
assertBusy(() -> {
|
||||
// We need to wait until all the records are processed from the internal execution queue, only then we can assert
|
||||
// that numRecords watch records have been processed as part of starting up.
|
||||
WatcherStatsResponse response = watcherClient().prepareWatcherStats().get();
|
||||
assertThat(response.getWatcherState(), equalTo(WatcherState.STARTED));
|
||||
assertThat(response.getThreadPoolQueueSize(), equalTo(0L));
|
||||
WatcherStatsResponse response = watcherClient().prepareWatcherStats().setIncludeCurrentWatches(true).get();
|
||||
long maxSize = response.getNodes().stream().map(WatcherStatsResponse.Node::getSnapshots).mapToLong(List::size).sum();
|
||||
assertThat(maxSize, equalTo(0L));
|
||||
|
||||
// because we try to execute a single watch in parallel, only one execution should happen
|
||||
refresh();
|
||||
SearchResponse searchResponse = client().prepareSearch("output").get();
|
||||
assertThat(searchResponse.getHits().getTotalHits(), is(greaterThanOrEqualTo(numberOfWatches)));
|
||||
|
@ -289,9 +290,9 @@ public class BootStrapTests extends AbstractWatcherIntegrationTestCase {
|
|||
|
||||
// the watch history should contain entries for each triggered watch, which a few have been marked as not executed
|
||||
SearchResponse historySearchResponse = client().prepareSearch(HistoryStore.INDEX_PREFIX + "*")
|
||||
.setSize(expectedWatchHistoryCount).get();
|
||||
.setSize(10000).get();
|
||||
assertHitCount(historySearchResponse, expectedWatchHistoryCount);
|
||||
long notExecutedCount = Arrays.asList(historySearchResponse.getHits().getHits()).stream()
|
||||
long notExecutedCount = Arrays.stream(historySearchResponse.getHits().getHits())
|
||||
.filter(hit -> hit.getSourceAsMap().get("state").equals(ExecutionState.NOT_EXECUTED_ALREADY_QUEUED.id()))
|
||||
.count();
|
||||
logger.info("Watches not executed: [{}]: expected watch history count [{}] - [{}] successful watch exections",
|
||||
|
@ -301,19 +302,15 @@ public class BootStrapTests extends AbstractWatcherIntegrationTestCase {
|
|||
}
|
||||
|
||||
public void testManuallyStopped() throws Exception {
|
||||
ensureWatcherStarted();
|
||||
WatcherStatsResponse response = watcherClient().prepareWatcherStats().get();
|
||||
assertThat(response.getWatcherMetaData().manuallyStopped(), is(false));
|
||||
assertThat(response.watcherMetaData().manuallyStopped(), is(false));
|
||||
stopWatcher();
|
||||
response = watcherClient().prepareWatcherStats().get();
|
||||
assertThat(response.getWatcherMetaData().manuallyStopped(), is(true));
|
||||
assertThat(response.watcherMetaData().manuallyStopped(), is(true));
|
||||
startWatcher();
|
||||
response = watcherClient().prepareWatcherStats().get();
|
||||
assertThat(response.getWatcherMetaData().manuallyStopped(), is(false));
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean enableSecurity() {
|
||||
return false;
|
||||
assertThat(response.watcherMetaData().manuallyStopped(), is(false));
|
||||
}
|
||||
|
||||
public void testWatchRecordSavedTwice() throws Exception {
|
||||
|
@ -327,32 +324,39 @@ public class BootStrapTests extends AbstractWatcherIntegrationTestCase {
|
|||
DateTime triggeredTime = new DateTime(2015, 11, 5, 0, 0, 0, 0, DateTimeZone.UTC);
|
||||
final String watchRecordIndex = HistoryStore.getHistoryIndexNameForTime(triggeredTime);
|
||||
|
||||
logger.info("Stopping watcher");
|
||||
ensureWatcherStarted();
|
||||
stopWatcher();
|
||||
|
||||
BulkRequestBuilder bulkRequestBuilder = client().prepareBulk();
|
||||
int numRecords = scaledRandomIntBetween(8, 32);
|
||||
for (int i = 0; i < numRecords; i++) {
|
||||
String watchId = Integer.toString(i);
|
||||
ScheduleTriggerEvent event = new ScheduleTriggerEvent(watchId, triggeredTime, triggeredTime);
|
||||
Wid wid = new Wid(watchId, triggeredTime);
|
||||
TriggeredWatch triggeredWatch = new TriggeredWatch(wid, event);
|
||||
client().prepareIndex(TriggeredWatchStore.INDEX_NAME, TriggeredWatchStore.DOC_TYPE, triggeredWatch.id().value())
|
||||
.setSource(jsonBuilder().value(triggeredWatch))
|
||||
.get();
|
||||
bulkRequestBuilder.add(
|
||||
client().prepareIndex(TriggeredWatchStore.INDEX_NAME, TriggeredWatchStore.DOC_TYPE, triggeredWatch.id().value())
|
||||
.setSource(jsonBuilder().value(triggeredWatch))
|
||||
);
|
||||
|
||||
WatchRecord watchRecord = new WatchRecord.MessageWatchRecord(wid, event, ExecutionState.EXECUTED, "executed");
|
||||
client().prepareIndex(watchRecordIndex, HistoryStore.DOC_TYPE, watchRecord.id().value())
|
||||
String id = internalCluster().getInstance(ClusterService.class).localNode().getId();
|
||||
WatchRecord watchRecord = new WatchRecord.MessageWatchRecord(wid, event, ExecutionState.EXECUTED, "executed", id);
|
||||
bulkRequestBuilder.add(client().prepareIndex(watchRecordIndex, HistoryStore.DOC_TYPE, watchRecord.id().value())
|
||||
.setSource(jsonBuilder().value(watchRecord))
|
||||
.get();
|
||||
|
||||
);
|
||||
}
|
||||
assertNoFailures(bulkRequestBuilder.setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE).get());
|
||||
|
||||
stopWatcher();
|
||||
logger.info("Starting watcher");
|
||||
startWatcher();
|
||||
|
||||
assertBusy(() -> {
|
||||
// We need to wait until all the records are processed from the internal execution queue, only then we can assert
|
||||
// that numRecords watch records have been processed as part of starting up.
|
||||
WatcherStatsResponse response = watcherClient().prepareWatcherStats().get();
|
||||
assertThat(response.getWatcherState(), equalTo(WatcherState.STARTED));
|
||||
assertThat(response.getThreadPoolQueueSize(), equalTo(0L));
|
||||
WatcherStatsResponse response = watcherClient().prepareWatcherStats().setIncludeCurrentWatches(true).get();
|
||||
long maxSize = response.getNodes().stream().map(WatcherStatsResponse.Node::getSnapshots).mapToLong(List::size).sum();
|
||||
assertThat(maxSize, equalTo(0L));
|
||||
|
||||
// but even then since the execution of the watch record is async it may take a little bit before
|
||||
// the actual documents are in the output index
|
||||
|
@ -361,7 +365,7 @@ public class BootStrapTests extends AbstractWatcherIntegrationTestCase {
|
|||
assertThat(searchResponse.getHits().getTotalHits(), Matchers.equalTo((long) numRecords));
|
||||
for (int i = 0; i < numRecords; i++) {
|
||||
assertThat(searchResponse.getHits().getAt(i).getSourceAsMap().get("state"),
|
||||
is(ExecutionState.EXECUTED_MULTIPLE_TIMES.id()));
|
||||
is(ExecutionState.EXECUTED.id()));
|
||||
}
|
||||
});
|
||||
}
|
||||
|
|
|
@ -130,7 +130,7 @@ public class ExecutionVarsIntegrationTests extends AbstractWatcherIntegrationTes
|
|||
|
||||
assertThat(putWatchResponse.isCreated(), is(true));
|
||||
|
||||
timeWarp().scheduler().trigger("_id");
|
||||
timeWarp().trigger("_id");
|
||||
|
||||
flush();
|
||||
refresh();
|
||||
|
|
|
@ -161,7 +161,7 @@ public class HipChatServiceTests extends AbstractWatcherIntegrationTestCase {
|
|||
|
||||
assertThat(putWatchResponse.isCreated(), is(true));
|
||||
|
||||
timeWarp().scheduler().trigger("1");
|
||||
timeWarp().trigger("1");
|
||||
flush();
|
||||
refresh();
|
||||
|
||||
|
|
|
@ -42,6 +42,12 @@ import static org.hamcrest.Matchers.nullValue;
|
|||
|
||||
public class HistoryIntegrationTests extends AbstractWatcherIntegrationTestCase {
|
||||
|
||||
// FOR TESTING
|
||||
@Override
|
||||
protected boolean enableSecurity() {
|
||||
return false;
|
||||
}
|
||||
|
||||
// issue: https://github.com/elastic/x-plugins/issues/2338
|
||||
public void testThatHistoryIsWrittenWithChainedInput() throws Exception {
|
||||
XContentBuilder xContentBuilder = jsonBuilder().startObject().startObject("inner").field("date", "2015-06-06").endObject()
|
||||
|
@ -90,6 +96,7 @@ public class HistoryIntegrationTests extends AbstractWatcherIntegrationTestCase
|
|||
.addAction("_logger", loggingAction("#### randomLogging")))
|
||||
.get();
|
||||
|
||||
ensureWatcherStarted();
|
||||
watcherClient().prepareExecuteWatch("test_watch").setRecordExecution(true).get();
|
||||
|
||||
flush(".watcher-history*");
|
||||
|
|
|
@ -31,6 +31,7 @@ import org.junit.Before;
|
|||
import java.util.Map;
|
||||
|
||||
import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
|
||||
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
|
||||
import static org.elasticsearch.xpack.watcher.actions.ActionBuilders.loggingAction;
|
||||
import static org.elasticsearch.xpack.watcher.actions.ActionBuilders.webhookAction;
|
||||
import static org.elasticsearch.xpack.watcher.client.WatchSourceBuilders.watchBuilder;
|
||||
|
@ -117,8 +118,8 @@ public class HttpSecretsIntegrationTests extends AbstractWatcherIntegrationTestC
|
|||
assertThat(value, nullValue()); // and yet we don't have the password
|
||||
|
||||
// now we restart, to make sure the watches and their secrets are reloaded from the index properly
|
||||
assertThat(watcherClient.prepareWatchService().restart().get().isAcknowledged(), is(true));
|
||||
ensureWatcherStarted();
|
||||
stopWatcher();
|
||||
startWatcher();
|
||||
|
||||
// now lets execute the watch manually
|
||||
|
||||
|
@ -182,8 +183,8 @@ public class HttpSecretsIntegrationTests extends AbstractWatcherIntegrationTestC
|
|||
assertThat(value, nullValue()); // and yet we don't have the password
|
||||
|
||||
// now we restart, to make sure the watches and their secrets are reloaded from the index properly
|
||||
assertThat(watcherClient.prepareWatchService().restart().get().isAcknowledged(), is(true));
|
||||
ensureWatcherStarted();
|
||||
stopWatcher();
|
||||
startWatcher();
|
||||
|
||||
// now lets execute the watch manually
|
||||
|
||||
|
|
|
@ -1,256 +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.xpack.watcher.test.integration;
|
||||
|
||||
import org.apache.lucene.util.LuceneTestCase.BadApple;
|
||||
import org.elasticsearch.ExceptionsHelper;
|
||||
import org.elasticsearch.client.Client;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.unit.TimeValue;
|
||||
import org.elasticsearch.discovery.Discovery;
|
||||
import org.elasticsearch.discovery.DiscoverySettings;
|
||||
import org.elasticsearch.discovery.MasterNotDiscoveredException;
|
||||
import org.elasticsearch.discovery.zen.ElectMasterService;
|
||||
import org.elasticsearch.discovery.zen.UnicastZenPing;
|
||||
import org.elasticsearch.discovery.zen.ZenPing;
|
||||
import org.elasticsearch.test.ESIntegTestCase.ClusterScope;
|
||||
import org.elasticsearch.test.discovery.ClusterDiscoveryConfiguration;
|
||||
import org.elasticsearch.test.discovery.TestZenDiscovery;
|
||||
import org.elasticsearch.test.junit.annotations.TestLogging;
|
||||
import org.elasticsearch.xpack.watcher.WatcherService;
|
||||
import org.elasticsearch.xpack.watcher.WatcherState;
|
||||
import org.elasticsearch.xpack.watcher.client.WatchSourceBuilder;
|
||||
import org.elasticsearch.xpack.watcher.client.WatchSourceBuilders;
|
||||
import org.elasticsearch.xpack.watcher.condition.AlwaysCondition;
|
||||
import org.elasticsearch.xpack.watcher.condition.CompareCondition;
|
||||
import org.elasticsearch.xpack.watcher.execution.ExecutionService;
|
||||
import org.elasticsearch.xpack.watcher.support.search.WatcherSearchTemplateRequest;
|
||||
import org.elasticsearch.xpack.watcher.test.AbstractWatcherIntegrationTestCase;
|
||||
import org.elasticsearch.xpack.watcher.transport.actions.delete.DeleteWatchResponse;
|
||||
import org.elasticsearch.xpack.watcher.transport.actions.stats.WatcherStatsResponse;
|
||||
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import static org.elasticsearch.index.query.QueryBuilders.termQuery;
|
||||
import static org.elasticsearch.search.builder.SearchSourceBuilder.searchSource;
|
||||
import static org.elasticsearch.test.ESIntegTestCase.Scope.TEST;
|
||||
import static org.elasticsearch.xpack.watcher.actions.ActionBuilders.loggingAction;
|
||||
import static org.elasticsearch.xpack.watcher.client.WatchSourceBuilders.watchBuilder;
|
||||
import static org.elasticsearch.xpack.watcher.input.InputBuilders.searchInput;
|
||||
import static org.elasticsearch.xpack.watcher.input.InputBuilders.simpleInput;
|
||||
import static org.elasticsearch.xpack.watcher.test.WatcherTestUtils.templateRequest;
|
||||
import static org.elasticsearch.xpack.watcher.trigger.TriggerBuilders.schedule;
|
||||
import static org.elasticsearch.xpack.watcher.trigger.schedule.Schedules.cron;
|
||||
import static org.elasticsearch.xpack.watcher.trigger.schedule.Schedules.interval;
|
||||
import static org.hamcrest.Matchers.equalTo;
|
||||
import static org.hamcrest.Matchers.instanceOf;
|
||||
import static org.hamcrest.core.Is.is;
|
||||
|
||||
//test is just too slow, please fix it to not be sleep-based
|
||||
@BadApple(bugUrl = "https://github.com/elastic/x-plugins/issues/1007")
|
||||
@TestLogging("org.elasticsearch.discovery:TRACE,org.elasticsearch.watcher:TRACE")
|
||||
@ClusterScope(scope = TEST, numClientNodes = 0, transportClientRatio = 0, randomDynamicTemplates = false, numDataNodes = 0,
|
||||
autoMinMasterNodes = false)
|
||||
public class NoMasterNodeTests extends AbstractWatcherIntegrationTestCase {
|
||||
private ClusterDiscoveryConfiguration.UnicastZen config;
|
||||
|
||||
@Override
|
||||
protected boolean timeWarped() {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean enableSecurity() {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Settings nodeSettings(int nodeOrdinal) {
|
||||
Settings settings = super.nodeSettings(nodeOrdinal);
|
||||
Settings unicastSettings = config.nodeSettings(nodeOrdinal);
|
||||
return Settings.builder()
|
||||
.put(settings)
|
||||
.put(unicastSettings)
|
||||
.put(ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey(), 2)
|
||||
.put(TestZenDiscovery.USE_MOCK_PINGS.getKey(), false)
|
||||
.build();
|
||||
}
|
||||
|
||||
public void testSimpleFailure() throws Exception {
|
||||
// we need 3 hosts here because we stop the master and start another - it doesn't restart the pre-existing node...
|
||||
config = new ClusterDiscoveryConfiguration.UnicastZen(3, Settings.EMPTY);
|
||||
internalCluster().startNodes(2);
|
||||
createIndex("my-index");
|
||||
ensureWatcherStarted(false);
|
||||
|
||||
// Have a sample document in the index, the watch is going to evaluate
|
||||
client().prepareIndex("my-index", "my-type").setSource("field", "value").get();
|
||||
WatcherSearchTemplateRequest request = templateRequest(searchSource().query(termQuery("field", "value")), "my-index");
|
||||
WatchSourceBuilder watchSource = watchBuilder()
|
||||
.trigger(schedule(cron("0/5 * * * * ? *")))
|
||||
.input(searchInput(request))
|
||||
.condition(new CompareCondition("ctx.payload.hits.total", CompareCondition.Op.EQ, 1L));
|
||||
|
||||
// we first need to make sure the license is enabled, otherwise all APIs will be blocked
|
||||
ensureLicenseEnabled();
|
||||
|
||||
watcherClient().preparePutWatch("my-first-watch")
|
||||
.setSource(watchSource)
|
||||
.get();
|
||||
assertWatchWithMinimumPerformedActionsCount("my-first-watch", 1);
|
||||
|
||||
// Stop the elected master, no new master will be elected b/c of m_m_n is set to 2
|
||||
stopElectedMasterNodeAndWait();
|
||||
try {
|
||||
// any watch action should fail, because there is no elected master node
|
||||
watcherClient().prepareDeleteWatch("my-first-watch").setMasterNodeTimeout(TimeValue.timeValueSeconds(1)).get();
|
||||
fail();
|
||||
} catch (Exception e) {
|
||||
assertThat(ExceptionsHelper.unwrapCause(e), instanceOf(MasterNotDiscoveredException.class));
|
||||
}
|
||||
// Bring back the 2nd node and wait for elected master node to come back and watcher to work as expected.
|
||||
startElectedMasterNodeAndWait();
|
||||
|
||||
// we first need to make sure the license is enabled, otherwise all APIs will be blocked
|
||||
ensureLicenseEnabled();
|
||||
|
||||
// Our first watch's condition should at least have been met twice
|
||||
assertWatchWithMinimumPerformedActionsCount("my-first-watch", 2);
|
||||
|
||||
// Delete the existing watch
|
||||
DeleteWatchResponse response = watcherClient().prepareDeleteWatch("my-first-watch").get();
|
||||
assertThat(response.isFound(), is(true));
|
||||
|
||||
// Add a new watch and wait for its condition to be met
|
||||
watcherClient().preparePutWatch("my-second-watch")
|
||||
.setSource(watchSource)
|
||||
.get();
|
||||
assertWatchWithMinimumPerformedActionsCount("my-second-watch", 1);
|
||||
}
|
||||
|
||||
public void testDedicatedMasterNodeLayout() throws Exception {
|
||||
// Only the master nodes are in the unicast nodes list:
|
||||
config = new ClusterDiscoveryConfiguration.UnicastZen(11, 3, Settings.EMPTY);
|
||||
internalCluster().startMasterOnlyNodes(3);
|
||||
internalCluster().startDataOnlyNodes(7);
|
||||
ensureWatcherStarted(false);
|
||||
ensureLicenseEnabled();
|
||||
|
||||
WatchSourceBuilder watchSource = WatchSourceBuilders.watchBuilder()
|
||||
.trigger(schedule(interval("5s")))
|
||||
.input(simpleInput("key", "value"))
|
||||
.condition(AlwaysCondition.INSTANCE)
|
||||
.addAction("_id", loggingAction("executed!"));
|
||||
|
||||
watcherClient().preparePutWatch("_watch_id")
|
||||
.setSource(watchSource)
|
||||
.get();
|
||||
assertWatchWithMinimumPerformedActionsCount("_watch_id", 1, false);
|
||||
|
||||
// We still have 2 master node, we should recover from this failure:
|
||||
internalCluster().stopCurrentMasterNode();
|
||||
ensureWatcherStarted(false);
|
||||
ensureWatcherOnlyRunningOnce();
|
||||
assertWatchWithMinimumPerformedActionsCount("_watch_id", 2, false);
|
||||
|
||||
// Stop the elected master, no new master will be elected b/c of m_m_n is set to 2
|
||||
stopElectedMasterNodeAndWait();
|
||||
try {
|
||||
// any watch action should fail, because there is no elected master node
|
||||
watcherClient().prepareDeleteWatch("_watch_id").setMasterNodeTimeout(TimeValue.timeValueSeconds(1)).get();
|
||||
fail();
|
||||
} catch (Exception e) {
|
||||
assertThat(ExceptionsHelper.unwrapCause(e), instanceOf(MasterNotDiscoveredException.class));
|
||||
}
|
||||
// Bring back the 2nd node and wait for elected master node to come back and watcher to work as expected.
|
||||
startElectedMasterNodeAndWait();
|
||||
|
||||
// we first need to make sure the license is enabled, otherwise all APIs will be blocked
|
||||
ensureLicenseEnabled();
|
||||
|
||||
// Our first watch's condition should at least have been met twice
|
||||
assertWatchWithMinimumPerformedActionsCount("_watch_id", 3, false);
|
||||
}
|
||||
|
||||
public void testMultipleFailures() throws Exception {
|
||||
int numberOfFailures = scaledRandomIntBetween(2, 9);
|
||||
int numberOfWatches = scaledRandomIntBetween(numberOfFailures, 12);
|
||||
logger.info("number of failures [{}], number of watches [{}]", numberOfFailures, numberOfWatches);
|
||||
config = new ClusterDiscoveryConfiguration.UnicastZen(2 + numberOfFailures, Settings.EMPTY);
|
||||
internalCluster().startNodes(2);
|
||||
createIndex("my-index");
|
||||
client().prepareIndex("my-index", "my-type").setSource("field", "value").get();
|
||||
|
||||
// watcher starts in the background, it can happen we get here too soon, so wait until watcher has started.
|
||||
ensureWatcherStarted(false);
|
||||
ensureLicenseEnabled();
|
||||
for (int i = 1; i <= numberOfWatches; i++) {
|
||||
String watchName = "watch" + i;
|
||||
WatcherSearchTemplateRequest request = templateRequest(searchSource().query(termQuery("field", "value")), "my-index");
|
||||
WatchSourceBuilder watchSource = watchBuilder()
|
||||
.trigger(schedule(cron("0/5 * * * * ? *")))
|
||||
.input(searchInput(request))
|
||||
.condition(new CompareCondition("ctx.payload.hits.total", CompareCondition.Op.EQ, 1L));
|
||||
watcherClient().preparePutWatch(watchName).setSource(watchSource).get();
|
||||
}
|
||||
ensureGreen();
|
||||
|
||||
for (int i = 1; i <= numberOfFailures; i++) {
|
||||
logger.info("failure round {}", i);
|
||||
|
||||
for (int j = 1; j < numberOfWatches; j++) {
|
||||
String watchName = "watch" + i;
|
||||
assertWatchWithMinimumPerformedActionsCount(watchName, i);
|
||||
}
|
||||
ensureGreen();
|
||||
stopElectedMasterNodeAndWait();
|
||||
startElectedMasterNodeAndWait();
|
||||
|
||||
WatcherStatsResponse statsResponse = watcherClient().prepareWatcherStats().get();
|
||||
assertThat(statsResponse.getWatchesCount(), equalTo((long) numberOfWatches));
|
||||
}
|
||||
}
|
||||
|
||||
private void stopElectedMasterNodeAndWait() throws Exception {
|
||||
// Due to the fact that the elected master node gets stopped before the ping timeout has passed,
|
||||
// the unicast ping may have cached ping responses. This can cause the existing node to think
|
||||
// that there is another node and if its node id is lower than the node that has been stopped it
|
||||
// will elect itself as master. This is bad and should be fixed in core. What I think that should happen is that
|
||||
// if a node detects that is has lost a node, a node should clear its unicast temporal responses or at least
|
||||
// remove the node that has been removed. This is a workaround:
|
||||
ZenPing zenPing = ((TestZenDiscovery)internalCluster().getInstance(Discovery.class)).getZenPing();
|
||||
if (zenPing instanceof UnicastZenPing) {
|
||||
((UnicastZenPing) zenPing).clearTemporalResponses();
|
||||
}
|
||||
internalCluster().stopCurrentMasterNode();
|
||||
// Can't use ensureWatcherStopped, b/c that relies on the watcher stats api which requires an elected master node
|
||||
assertBusy(new Runnable() {
|
||||
@Override
|
||||
public void run () {
|
||||
for (Client client : clients()) {
|
||||
ClusterState state = client.admin().cluster().prepareState().setLocal(true).get().getState();
|
||||
assertThat("Node [" + state.nodes().getLocalNode() + "] should have a NO_MASTER_BLOCK",
|
||||
state.blocks().hasGlobalBlock(DiscoverySettings.NO_MASTER_BLOCK_ID), is(true));
|
||||
}
|
||||
}
|
||||
}, 30, TimeUnit.SECONDS);
|
||||
// Ensure that the watch manager doesn't run elsewhere
|
||||
for (WatcherService watcherService : internalCluster().getInstances(WatcherService.class)) {
|
||||
assertThat(watcherService.state(), is(WatcherState.STOPPED));
|
||||
}
|
||||
for (ExecutionService executionService : internalCluster().getInstances(ExecutionService.class)) {
|
||||
assertThat(executionService.executionThreadPoolQueueSize(), equalTo(0L));
|
||||
}
|
||||
}
|
||||
|
||||
private void startElectedMasterNodeAndWait() throws Exception {
|
||||
internalCluster().startNode();
|
||||
ensureWatcherStarted(false);
|
||||
ensureWatcherOnlyRunningOnce();
|
||||
}
|
||||
|
||||
}
|
|
@ -88,7 +88,7 @@ public class PagerDutyServiceTests extends AbstractWatcherIntegrationTestCase {
|
|||
|
||||
assertThat(putWatchResponse.isCreated(), is(true));
|
||||
|
||||
timeWarp().scheduler().trigger("1");
|
||||
timeWarp().trigger("1");
|
||||
flush();
|
||||
refresh();
|
||||
|
||||
|
|
|
@ -109,7 +109,7 @@ public class SlackServiceTests extends AbstractWatcherIntegrationTestCase {
|
|||
|
||||
assertThat(putWatchResponse.isCreated(), is(true));
|
||||
|
||||
timeWarp().scheduler().trigger("1");
|
||||
timeWarp().trigger("1");
|
||||
flush();
|
||||
refresh();
|
||||
|
||||
|
|
|
@ -14,8 +14,8 @@ import org.elasticsearch.action.index.IndexResponse;
|
|||
import org.elasticsearch.action.support.WriteRequest;
|
||||
import org.elasticsearch.common.unit.TimeValue;
|
||||
import org.elasticsearch.common.xcontent.XContentType;
|
||||
import org.elasticsearch.test.junit.annotations.TestLogging;
|
||||
import org.elasticsearch.xpack.watcher.actions.ActionStatus;
|
||||
import org.elasticsearch.xpack.watcher.client.WatcherClient;
|
||||
import org.elasticsearch.xpack.watcher.condition.CompareCondition;
|
||||
import org.elasticsearch.xpack.watcher.execution.ExecutionState;
|
||||
import org.elasticsearch.xpack.watcher.history.HistoryStore;
|
||||
|
@ -30,7 +30,6 @@ import org.elasticsearch.xpack.watcher.watch.Watch;
|
|||
import org.hamcrest.Matchers;
|
||||
import org.junit.Before;
|
||||
|
||||
import java.util.UUID;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery;
|
||||
|
@ -48,10 +47,10 @@ import static org.hamcrest.Matchers.greaterThanOrEqualTo;
|
|||
import static org.hamcrest.Matchers.is;
|
||||
import static org.hamcrest.core.IsEqual.equalTo;
|
||||
|
||||
@TestLogging("org.elasticsearch.xpack.watcher:TRACE")
|
||||
public class WatchAckTests extends AbstractWatcherIntegrationTestCase {
|
||||
|
||||
private WatcherClient watcherClient;
|
||||
private UUID id = UUID.randomUUID();
|
||||
private String id = randomAlphaOfLength(10);
|
||||
|
||||
@Override
|
||||
protected boolean timeWarped() {
|
||||
|
@ -65,8 +64,7 @@ public class WatchAckTests extends AbstractWatcherIntegrationTestCase {
|
|||
|
||||
@Before
|
||||
public void indexTestDocument() {
|
||||
watcherClient = watcherClient();
|
||||
IndexResponse eventIndexResponse = client().prepareIndex("events", "event", id.toString())
|
||||
IndexResponse eventIndexResponse = client().prepareIndex("events", "event", id)
|
||||
.setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE)
|
||||
.setSource("level", "error")
|
||||
.get();
|
||||
|
@ -74,7 +72,7 @@ public class WatchAckTests extends AbstractWatcherIntegrationTestCase {
|
|||
}
|
||||
|
||||
public void testAckSingleAction() throws Exception {
|
||||
PutWatchResponse putWatchResponse = watcherClient.preparePutWatch()
|
||||
PutWatchResponse putWatchResponse = watcherClient().preparePutWatch()
|
||||
.setId("_id")
|
||||
.setSource(watchBuilder()
|
||||
.trigger(schedule(cron("0/5 * * * * ? *")))
|
||||
|
@ -87,9 +85,10 @@ public class WatchAckTests extends AbstractWatcherIntegrationTestCase {
|
|||
.get();
|
||||
|
||||
assertThat(putWatchResponse.isCreated(), is(true));
|
||||
assertThat(watcherClient().prepareWatcherStats().get().getWatchesCount(), is(1L));
|
||||
|
||||
timeWarp().scheduler().trigger("_id", 4, TimeValue.timeValueSeconds(5));
|
||||
AckWatchResponse ackResponse = watcherClient.prepareAckWatch("_id").setActionIds("_a1").get();
|
||||
timeWarp().trigger("_id", 4, TimeValue.timeValueSeconds(5));
|
||||
AckWatchResponse ackResponse = watcherClient().prepareAckWatch("_id").setActionIds("_a1").get();
|
||||
assertThat(ackResponse.getStatus().actionStatus("_a1").ackStatus().state(), is(ActionStatus.AckStatus.State.ACKED));
|
||||
assertThat(ackResponse.getStatus().actionStatus("_a2").ackStatus().state(), is(ActionStatus.AckStatus.State.ACKABLE));
|
||||
|
||||
|
@ -99,7 +98,7 @@ public class WatchAckTests extends AbstractWatcherIntegrationTestCase {
|
|||
assertThat(a1CountAfterAck, greaterThan(0L));
|
||||
assertThat(a2CountAfterAck, greaterThan(0L));
|
||||
|
||||
timeWarp().scheduler().trigger("_id", 4, TimeValue.timeValueSeconds(5));
|
||||
timeWarp().trigger("_id", 4, TimeValue.timeValueSeconds(5));
|
||||
flush();
|
||||
refresh();
|
||||
|
||||
|
@ -112,13 +111,13 @@ public class WatchAckTests extends AbstractWatcherIntegrationTestCase {
|
|||
assertThat(a2CountAfterPostAckFires, greaterThan(a2CountAfterAck));
|
||||
|
||||
// Now delete the event and the ack states should change to AWAITS_EXECUTION
|
||||
DeleteResponse response = client().prepareDelete("events", "event", id.toString()).get();
|
||||
DeleteResponse response = client().prepareDelete("events", "event", id).get();
|
||||
assertEquals(DocWriteResponse.Result.DELETED, response.getResult());
|
||||
refresh();
|
||||
|
||||
timeWarp().scheduler().trigger("_id", 4, TimeValue.timeValueSeconds(5));
|
||||
timeWarp().trigger("_id", 4, TimeValue.timeValueSeconds(5));
|
||||
|
||||
GetWatchResponse getWatchResponse = watcherClient.prepareGetWatch("_id").get();
|
||||
GetWatchResponse getWatchResponse = watcherClient().prepareGetWatch("_id").get();
|
||||
assertThat(getWatchResponse.isFound(), is(true));
|
||||
|
||||
Watch parsedWatch = watchParser().parse(getWatchResponse.getId(), true, getWatchResponse.getSource().getBytes(), XContentType.JSON);
|
||||
|
@ -133,7 +132,7 @@ public class WatchAckTests extends AbstractWatcherIntegrationTestCase {
|
|||
}
|
||||
|
||||
public void testAckAllActions() throws Exception {
|
||||
PutWatchResponse putWatchResponse = watcherClient.preparePutWatch()
|
||||
PutWatchResponse putWatchResponse = watcherClient().preparePutWatch()
|
||||
.setId("_id")
|
||||
.setSource(watchBuilder()
|
||||
.trigger(schedule(cron("0/5 * * * * ? *")))
|
||||
|
@ -146,10 +145,11 @@ public class WatchAckTests extends AbstractWatcherIntegrationTestCase {
|
|||
.get();
|
||||
|
||||
assertThat(putWatchResponse.isCreated(), is(true));
|
||||
assertThat(watcherClient().prepareWatcherStats().get().getWatchesCount(), is(1L));
|
||||
|
||||
timeWarp().scheduler().trigger("_id", 4, TimeValue.timeValueSeconds(5));
|
||||
timeWarp().trigger("_id", 4, TimeValue.timeValueSeconds(5));
|
||||
|
||||
AckWatchRequestBuilder ackWatchRequestBuilder = watcherClient.prepareAckWatch("_id");
|
||||
AckWatchRequestBuilder ackWatchRequestBuilder = watcherClient().prepareAckWatch("_id");
|
||||
if (randomBoolean()) {
|
||||
ackWatchRequestBuilder.setActionIds("_all");
|
||||
} else if (randomBoolean()) {
|
||||
|
@ -166,7 +166,7 @@ public class WatchAckTests extends AbstractWatcherIntegrationTestCase {
|
|||
assertThat(a1CountAfterAck, greaterThanOrEqualTo((long) 1));
|
||||
assertThat(a2CountAfterAck, greaterThanOrEqualTo((long) 1));
|
||||
|
||||
timeWarp().scheduler().trigger("_id", 4, TimeValue.timeValueSeconds(5));
|
||||
timeWarp().trigger("_id", 4, TimeValue.timeValueSeconds(5));
|
||||
flush();
|
||||
refresh();
|
||||
|
||||
|
@ -179,13 +179,13 @@ public class WatchAckTests extends AbstractWatcherIntegrationTestCase {
|
|||
assertThat(a2CountAfterPostAckFires, equalTo(a2CountAfterAck));
|
||||
|
||||
// Now delete the event and the ack states should change to AWAITS_EXECUTION
|
||||
DeleteResponse response = client().prepareDelete("events", "event", id.toString()).get();
|
||||
DeleteResponse response = client().prepareDelete("events", "event", id).get();
|
||||
assertEquals(DocWriteResponse.Result.DELETED, response.getResult());
|
||||
refresh();
|
||||
|
||||
timeWarp().scheduler().trigger("_id", 4, TimeValue.timeValueSeconds(5));
|
||||
timeWarp().trigger("_id", 4, TimeValue.timeValueSeconds(5));
|
||||
|
||||
GetWatchResponse getWatchResponse = watcherClient.prepareGetWatch("_id").get();
|
||||
GetWatchResponse getWatchResponse = watcherClient().prepareGetWatch("_id").get();
|
||||
assertThat(getWatchResponse.isFound(), is(true));
|
||||
|
||||
Watch parsedWatch = watchParser().parse(getWatchResponse.getId(), true, getWatchResponse.getSource().getBytes(), XContentType.JSON);
|
||||
|
@ -200,7 +200,7 @@ public class WatchAckTests extends AbstractWatcherIntegrationTestCase {
|
|||
}
|
||||
|
||||
public void testAckWithRestart() throws Exception {
|
||||
PutWatchResponse putWatchResponse = watcherClient.preparePutWatch()
|
||||
PutWatchResponse putWatchResponse = watcherClient().preparePutWatch()
|
||||
.setId("_name")
|
||||
.setSource(watchBuilder()
|
||||
.trigger(schedule(cron("0/5 * * * * ? *")))
|
||||
|
@ -210,11 +210,12 @@ public class WatchAckTests extends AbstractWatcherIntegrationTestCase {
|
|||
.addAction("_id", indexAction("actions", "action")))
|
||||
.get();
|
||||
assertThat(putWatchResponse.isCreated(), is(true));
|
||||
assertThat(watcherClient().prepareWatcherStats().get().getWatchesCount(), is(1L));
|
||||
|
||||
timeWarp().scheduler().trigger("_name", 4, TimeValue.timeValueSeconds(5));
|
||||
timeWarp().trigger("_name", 4, TimeValue.timeValueSeconds(5));
|
||||
restartWatcherRandomly();
|
||||
|
||||
AckWatchResponse ackResponse = watcherClient.prepareAckWatch("_name").get();
|
||||
AckWatchResponse ackResponse = watcherClient().prepareAckWatch("_name").get();
|
||||
assertThat(ackResponse.getStatus().actionStatus("_id").ackStatus().state(), is(ActionStatus.AckStatus.State.ACKED));
|
||||
|
||||
refresh("actions");
|
||||
|
@ -223,7 +224,7 @@ public class WatchAckTests extends AbstractWatcherIntegrationTestCase {
|
|||
|
||||
restartWatcherRandomly();
|
||||
|
||||
GetWatchResponse watchResponse = watcherClient.getWatch(new GetWatchRequest("_name")).actionGet();
|
||||
GetWatchResponse watchResponse = watcherClient().getWatch(new GetWatchRequest("_name")).actionGet();
|
||||
assertThat(watchResponse.getStatus().actionStatus("_id").ackStatus().state(), Matchers.equalTo(ActionStatus.AckStatus.State.ACKED));
|
||||
|
||||
refresh();
|
||||
|
@ -232,7 +233,7 @@ public class WatchAckTests extends AbstractWatcherIntegrationTestCase {
|
|||
assertThat(watchResponse.getStatus().actionStatus("_id").ackStatus().state(),
|
||||
equalTo(indexedWatch.status().actionStatus("_id").ackStatus().state()));
|
||||
|
||||
timeWarp().scheduler().trigger("_name", 4, TimeValue.timeValueSeconds(5));
|
||||
timeWarp().trigger("_name", 4, TimeValue.timeValueSeconds(5));
|
||||
refresh("actions");
|
||||
|
||||
// There shouldn't be more actions in the index after we ack the watch, even though the watch was triggered
|
||||
|
@ -242,6 +243,7 @@ public class WatchAckTests extends AbstractWatcherIntegrationTestCase {
|
|||
|
||||
private void restartWatcherRandomly() throws Exception {
|
||||
if (randomBoolean()) {
|
||||
ensureWatcherStarted();
|
||||
stopWatcher();
|
||||
ensureWatcherStopped();
|
||||
startWatcher();
|
||||
|
|
|
@ -59,7 +59,7 @@ public class WatchMetadataTests extends AbstractWatcherIntegrationTestCase {
|
|||
.get();
|
||||
|
||||
if (timeWarped()) {
|
||||
timeWarp().scheduler().trigger("_name");
|
||||
timeWarp().trigger("_name");
|
||||
} else {
|
||||
// Wait for a no action entry to be added. (the condition search request will not match, because there are no docs in my-index)
|
||||
assertWatchWithNoActionNeeded("_name", 1);
|
||||
|
|
|
@ -17,6 +17,7 @@ import org.elasticsearch.plugins.Plugin;
|
|||
import org.elasticsearch.script.MockScriptPlugin;
|
||||
import org.elasticsearch.script.Script;
|
||||
import org.elasticsearch.script.ScriptType;
|
||||
import org.elasticsearch.test.junit.annotations.TestLogging;
|
||||
import org.elasticsearch.xpack.watcher.condition.AlwaysCondition;
|
||||
import org.elasticsearch.xpack.watcher.support.search.WatcherSearchTemplateRequest;
|
||||
import org.elasticsearch.xpack.watcher.test.AbstractWatcherIntegrationTestCase;
|
||||
|
@ -52,8 +53,19 @@ import static org.hamcrest.Matchers.equalTo;
|
|||
import static org.hamcrest.Matchers.greaterThanOrEqualTo;
|
||||
import static org.hamcrest.Matchers.is;
|
||||
|
||||
@TestLogging("org.elasticsearch.xpack.watcher:DEBUG,org.elasticsearch.xpack.watcher.WatcherIndexingListener:TRACE")
|
||||
public class TransformIntegrationTests extends AbstractWatcherIntegrationTestCase {
|
||||
|
||||
@Override
|
||||
protected boolean timeWarped() {
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean enableSecurity() {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Collection<Class<? extends Plugin>> nodePlugins() {
|
||||
List<Class<? extends Plugin>> types = super.pluginTypes();
|
||||
|
@ -152,8 +164,8 @@ public class TransformIntegrationTests extends AbstractWatcherIntegrationTestCas
|
|||
assertThat(putWatchResponse.isCreated(), is(true));
|
||||
|
||||
if (timeWarped()) {
|
||||
timeWarp().scheduler().trigger("_id1");
|
||||
timeWarp().scheduler().trigger("_id2");
|
||||
timeWarp().trigger("_id1");
|
||||
timeWarp().trigger("_id2");
|
||||
refresh();
|
||||
}
|
||||
|
||||
|
@ -201,8 +213,8 @@ public class TransformIntegrationTests extends AbstractWatcherIntegrationTestCas
|
|||
assertThat(putWatchResponse.isCreated(), is(true));
|
||||
|
||||
if (timeWarped()) {
|
||||
timeWarp().scheduler().trigger("_id1");
|
||||
timeWarp().scheduler().trigger("_id2");
|
||||
timeWarp().trigger("_id1");
|
||||
timeWarp().trigger("_id2");
|
||||
refresh();
|
||||
}
|
||||
|
||||
|
@ -247,8 +259,8 @@ public class TransformIntegrationTests extends AbstractWatcherIntegrationTestCas
|
|||
assertThat(putWatchResponse.isCreated(), is(true));
|
||||
|
||||
if (timeWarped()) {
|
||||
timeWarp().scheduler().trigger("_id1");
|
||||
timeWarp().scheduler().trigger("_id2");
|
||||
timeWarp().trigger("_id1");
|
||||
timeWarp().trigger("_id2");
|
||||
refresh();
|
||||
}
|
||||
|
||||
|
|
|
@ -14,6 +14,7 @@ import org.elasticsearch.common.util.set.Sets;
|
|||
import org.elasticsearch.common.xcontent.ToXContent;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.common.xcontent.XContentType;
|
||||
import org.elasticsearch.test.junit.annotations.TestLogging;
|
||||
import org.elasticsearch.xpack.watcher.client.WatcherClient;
|
||||
import org.elasticsearch.xpack.watcher.execution.ExecutionState;
|
||||
import org.elasticsearch.xpack.watcher.support.xcontent.XContentSource;
|
||||
|
@ -21,7 +22,9 @@ import org.elasticsearch.xpack.watcher.test.AbstractWatcherIntegrationTestCase;
|
|||
import org.elasticsearch.xpack.watcher.transport.actions.activate.ActivateWatchResponse;
|
||||
import org.elasticsearch.xpack.watcher.transport.actions.get.GetWatchResponse;
|
||||
import org.elasticsearch.xpack.watcher.transport.actions.put.PutWatchResponse;
|
||||
import org.elasticsearch.xpack.watcher.transport.actions.stats.WatcherStatsResponse;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
|
@ -36,6 +39,7 @@ import static org.hamcrest.Matchers.greaterThan;
|
|||
import static org.hamcrest.Matchers.is;
|
||||
import static org.hamcrest.Matchers.notNullValue;
|
||||
|
||||
@TestLogging("org.elasticsearch.xpack.watcher:DEBUG,org.elasticsearch.xpack.watcher.WatcherIndexingListener:TRACE")
|
||||
public class ActivateWatchTests extends AbstractWatcherIntegrationTestCase {
|
||||
|
||||
@Override
|
||||
|
@ -43,12 +47,14 @@ public class ActivateWatchTests extends AbstractWatcherIntegrationTestCase {
|
|||
return false;
|
||||
}
|
||||
|
||||
@AwaitsFix(bugUrl="https://github.com/elastic/x-pack-elasticsearch/issues/395")
|
||||
@Override
|
||||
protected boolean enableSecurity() {
|
||||
return false;
|
||||
}
|
||||
|
||||
// FIXME not to be sleep based
|
||||
public void testDeactivateAndActivate() throws Exception {
|
||||
WatcherClient watcherClient = watcherClient();
|
||||
|
||||
PutWatchResponse putWatchResponse = watcherClient.preparePutWatch()
|
||||
PutWatchResponse putWatchResponse = watcherClient().preparePutWatch()
|
||||
.setId("_id")
|
||||
.setSource(watchBuilder()
|
||||
.trigger(schedule(interval("1s")))
|
||||
|
@ -59,44 +65,55 @@ public class ActivateWatchTests extends AbstractWatcherIntegrationTestCase {
|
|||
|
||||
assertThat(putWatchResponse.isCreated(), is(true));
|
||||
|
||||
GetWatchResponse getWatchResponse = watcherClient.prepareGetWatch("_id").get();
|
||||
GetWatchResponse getWatchResponse = watcherClient().prepareGetWatch("_id").get();
|
||||
assertThat(getWatchResponse, notNullValue());
|
||||
assertThat(getWatchResponse.getStatus().state().isActive(), is(true));
|
||||
|
||||
assertWatchWithMinimumActionsCount("_id", ExecutionState.EXECUTED, 5);
|
||||
logger.info("Waiting for watch to be executed at least once");
|
||||
assertWatchWithMinimumActionsCount("_id", ExecutionState.EXECUTED, 1);
|
||||
|
||||
// we now know the watch is executing... lets deactivate it
|
||||
ActivateWatchResponse activateWatchResponse = watcherClient.prepareActivateWatch("_id", false).get();
|
||||
ActivateWatchResponse activateWatchResponse = watcherClient().prepareActivateWatch("_id", false).get();
|
||||
assertThat(activateWatchResponse, notNullValue());
|
||||
assertThat(activateWatchResponse.getStatus().state().isActive(), is(false));
|
||||
|
||||
getWatchResponse = watcherClient.prepareGetWatch("_id").get();
|
||||
getWatchResponse = watcherClient().prepareGetWatch("_id").get();
|
||||
assertThat(getWatchResponse, notNullValue());
|
||||
assertThat(getWatchResponse.getStatus().state().isActive(), is(false));
|
||||
|
||||
flush();
|
||||
// wait until no watch is executing
|
||||
assertBusy(() -> {
|
||||
WatcherStatsResponse statsResponse = watcherClient().prepareWatcherStats().setIncludeCurrentWatches(true).get();
|
||||
int sum = statsResponse.getNodes().stream().map(WatcherStatsResponse.Node::getSnapshots).mapToInt(List::size).sum();
|
||||
assertThat(sum, is(0));
|
||||
});
|
||||
|
||||
logger.info("Ensured no more watches are being executed");
|
||||
refresh();
|
||||
long count1 = docCount(".watcher-history*", "watch_record", matchAllQuery());
|
||||
|
||||
Thread.sleep(10000);
|
||||
logger.info("Sleeping for 5 seconds, watch history count [{}]", count1);
|
||||
Thread.sleep(5000);
|
||||
|
||||
flush();
|
||||
refresh();
|
||||
long count2 = docCount(".watcher-history*", "watch_record", matchAllQuery());
|
||||
|
||||
assertThat(count2, is(count1));
|
||||
|
||||
// lets activate it again
|
||||
logger.info("Activating watch again");
|
||||
|
||||
activateWatchResponse = watcherClient.prepareActivateWatch("_id", true).get();
|
||||
activateWatchResponse = watcherClient().prepareActivateWatch("_id", true).get();
|
||||
assertThat(activateWatchResponse, notNullValue());
|
||||
assertThat(activateWatchResponse.getStatus().state().isActive(), is(true));
|
||||
|
||||
getWatchResponse = watcherClient.prepareGetWatch("_id").get();
|
||||
getWatchResponse = watcherClient().prepareGetWatch("_id").get();
|
||||
assertThat(getWatchResponse, notNullValue());
|
||||
assertThat(getWatchResponse.getStatus().state().isActive(), is(true));
|
||||
|
||||
Thread.sleep(10000);
|
||||
logger.info("Sleeping for another five seconds, ensuring that watch is executed");
|
||||
Thread.sleep(5000);
|
||||
refresh();
|
||||
long count3 = docCount(".watcher-history*", "watch_record", matchAllQuery());
|
||||
assertThat(count3, greaterThan(count1));
|
||||
}
|
||||
|
@ -145,10 +162,9 @@ public class ActivateWatchTests extends AbstractWatcherIntegrationTestCase {
|
|||
assertThat(indexResponse.getId(), is("_id"));
|
||||
|
||||
// now, let's restart
|
||||
assertThat(watcherClient.prepareWatchService().stop().get().isAcknowledged(), is(true));
|
||||
ensureWatcherStopped();
|
||||
assertThat(watcherClient.prepareWatchService().start().get().isAcknowledged(), is(true));
|
||||
ensureWatcherStarted();
|
||||
stopWatcher();
|
||||
startWatcher();
|
||||
|
||||
getWatchResponse = watcherClient.prepareGetWatch("_id").get();
|
||||
assertThat(getWatchResponse, notNullValue());
|
||||
|
|
|
@ -265,15 +265,12 @@ public class ExecuteWatchTests extends AbstractWatcherIntegrationTestCase {
|
|||
|
||||
// lets wait for the watch to be ackable
|
||||
if (timeWarped()) {
|
||||
timeWarp().scheduler().trigger("_id");
|
||||
timeWarp().trigger("_id");
|
||||
} else {
|
||||
assertBusy(new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
GetWatchResponse getWatchResponse = watcherClient.prepareGetWatch("_id").get();
|
||||
assertThat(getWatchResponse.getStatus().actionStatus("log").ackStatus().state(),
|
||||
equalTo(ActionStatus.AckStatus.State.ACKABLE));
|
||||
}
|
||||
assertBusy(() -> {
|
||||
GetWatchResponse getWatchResponse = watcherClient.prepareGetWatch("_id").get();
|
||||
assertThat(getWatchResponse.getStatus().actionStatus("log").ackStatus().state(),
|
||||
equalTo(ActionStatus.AckStatus.State.ACKABLE));
|
||||
});
|
||||
}
|
||||
|
||||
|
|
|
@ -5,7 +5,6 @@
|
|||
*/
|
||||
package org.elasticsearch.xpack.watcher.transport.action.get;
|
||||
|
||||
import org.elasticsearch.action.admin.indices.alias.get.GetAliasesResponse;
|
||||
import org.elasticsearch.index.IndexNotFoundException;
|
||||
import org.elasticsearch.xpack.watcher.condition.AlwaysCondition;
|
||||
import org.elasticsearch.xpack.watcher.support.xcontent.XContentSource;
|
||||
|
@ -31,6 +30,7 @@ import static org.hamcrest.Matchers.notNullValue;
|
|||
import static org.hamcrest.Matchers.nullValue;
|
||||
|
||||
public class GetWatchTests extends AbstractWatcherIntegrationTestCase {
|
||||
|
||||
public void testGet() throws Exception {
|
||||
PutWatchResponse putResponse = watcherClient().preparePutWatch("_name").setSource(watchBuilder()
|
||||
.trigger(schedule(interval("5m")))
|
||||
|
@ -58,17 +58,13 @@ public class GetWatchTests extends AbstractWatcherIntegrationTestCase {
|
|||
public void testGetNotFoundOnNonExistingIndex() throws Exception {
|
||||
// ensure index/alias is deleted, test infra might have created it automatically
|
||||
try {
|
||||
client().admin().indices().prepareDelete(Watch.INDEX).get();
|
||||
assertAcked(client().admin().indices().prepareDelete(Watch.INDEX));
|
||||
} catch (IndexNotFoundException e) {}
|
||||
Exception e = expectThrows(Exception.class, () -> watcherClient().getWatch(new GetWatchRequest("_name")).get());
|
||||
assertThat(e.getMessage(), containsString("no such index"));
|
||||
}
|
||||
|
||||
public void testGetNotFound() throws Exception {
|
||||
GetAliasesResponse aliasesResponse = client().admin().indices().prepareGetAliases(Watch.INDEX).get();
|
||||
if (aliasesResponse.getAliases().isEmpty()) {
|
||||
assertAcked(client().admin().indices().prepareCreate(Watch.INDEX));
|
||||
}
|
||||
GetWatchResponse getResponse = watcherClient().getWatch(new GetWatchRequest("_name")).get();
|
||||
assertThat(getResponse, notNullValue());
|
||||
assertThat(getResponse.getId(), is("_name"));
|
||||
|
|
|
@ -16,6 +16,7 @@ import org.elasticsearch.xpack.watcher.actions.ActionBuilders;
|
|||
import org.elasticsearch.xpack.watcher.condition.ScriptCondition;
|
||||
import org.elasticsearch.xpack.watcher.execution.ExecutionPhase;
|
||||
import org.elasticsearch.xpack.watcher.execution.QueuedWatch;
|
||||
import org.elasticsearch.xpack.watcher.execution.WatchExecutionSnapshot;
|
||||
import org.elasticsearch.xpack.watcher.input.InputBuilders;
|
||||
import org.elasticsearch.xpack.watcher.test.AbstractWatcherIntegrationTestCase;
|
||||
import org.elasticsearch.xpack.watcher.transport.actions.stats.WatcherStatsResponse;
|
||||
|
@ -35,9 +36,10 @@ import static org.elasticsearch.xpack.watcher.trigger.schedule.Schedules.interva
|
|||
import static org.hamcrest.Matchers.anyOf;
|
||||
import static org.hamcrest.Matchers.equalTo;
|
||||
import static org.hamcrest.Matchers.greaterThanOrEqualTo;
|
||||
import static org.hamcrest.Matchers.hasSize;
|
||||
import static org.hamcrest.Matchers.lessThanOrEqualTo;
|
||||
import static org.hamcrest.Matchers.notNullValue;
|
||||
import static org.hamcrest.Matchers.nullValue;
|
||||
import static org.hamcrest.core.Is.is;
|
||||
|
||||
@ESIntegTestCase.ClusterScope(scope = SUITE, numClientNodes = 0, transportClientRatio = 0,
|
||||
randomDynamicTemplates = false, numDataNodes = 1, supportsDedicatedMasters = false)
|
||||
|
@ -79,7 +81,8 @@ public class WatchStatsTests extends AbstractWatcherIntegrationTestCase {
|
|||
getLatchScriptEngine().finishScriptExecution();
|
||||
}
|
||||
|
||||
@TestLogging("org.elasticsearch.xpack.watcher.trigger.schedule.engine:TRACE,org.elasticsearch.xpack.scheduler:TRACE")
|
||||
@TestLogging("org.elasticsearch.xpack.watcher.trigger.schedule.engine:TRACE,org.elasticsearch.xpack.scheduler:TRACE,org.elasticsearch" +
|
||||
".xpack.watcher.execution:TRACE,org.elasticsearch.xpack.watcher.test:TRACE")
|
||||
public void testCurrentWatches() throws Exception {
|
||||
watcherClient().preparePutWatch("_id").setSource(watchBuilder()
|
||||
.trigger(schedule(interval("1s")))
|
||||
|
@ -88,26 +91,33 @@ public class WatchStatsTests extends AbstractWatcherIntegrationTestCase {
|
|||
.addAction("_action", ActionBuilders.loggingAction("some logging"))
|
||||
).get();
|
||||
|
||||
logger.info("Waiting for first script invocation");
|
||||
getLatchScriptEngine().awaitScriptStartedExecution();
|
||||
logger.info("First script got executed, checking currently running watches");
|
||||
|
||||
WatcherStatsResponse response = watcherClient().prepareWatcherStats().setIncludeCurrentWatches(true).get();
|
||||
assertThat(response.getWatcherState(), equalTo(WatcherState.STARTED));
|
||||
boolean watcherStarted = response.getNodes().stream().anyMatch(node -> node.getWatcherState() == WatcherState.STARTED);
|
||||
assertThat(watcherStarted, is(true));
|
||||
|
||||
assertThat(response.getWatchesCount(), equalTo(1L));
|
||||
assertThat(response.getQueuedWatches(), nullValue());
|
||||
assertThat(response.getSnapshots(), notNullValue());
|
||||
assertThat(response.getSnapshots().size(), equalTo(1));
|
||||
assertThat(response.getSnapshots().get(0).watchId(), equalTo("_id"));
|
||||
assertThat(response.getSnapshots().get(0).executionPhase(), equalTo(ExecutionPhase.CONDITION));
|
||||
assertThat(getQueuedWatches(response), hasSize(0));
|
||||
List<WatchExecutionSnapshot> snapshots = getSnapshots(response);
|
||||
assertThat(snapshots, notNullValue());
|
||||
assertThat(snapshots, hasSize(1));
|
||||
assertThat(snapshots.get(0).watchId(), equalTo("_id"));
|
||||
assertThat(snapshots.get(0).executionPhase(), equalTo(ExecutionPhase.CONDITION));
|
||||
}
|
||||
|
||||
@TestLogging("org.elasticsearch.xpack.watcher.trigger.schedule.engine:TRACE,org.elasticsearch.xpack.scheduler:TRACE,org.elasticsearch" +
|
||||
".xpack.watcher.execution:TRACE,org.elasticsearch.xpack.watcher.test:TRACE")
|
||||
public void testPendingWatches() throws Exception {
|
||||
// Add 5 slow watches and we should almost immediately see pending watches in the stats api
|
||||
for (int i = 0; i < 5; i++) {
|
||||
watcherClient().preparePutWatch("_id" + i).setSource(watchBuilder()
|
||||
.trigger(schedule(interval("1s")))
|
||||
.input(noneInput())
|
||||
.condition(new ScriptCondition(LatchScriptEngine.latchScript()))
|
||||
.addAction("_action", ActionBuilders.loggingAction("some logging"))
|
||||
.trigger(schedule(interval("1s")))
|
||||
.input(noneInput())
|
||||
.condition(new ScriptCondition(LatchScriptEngine.latchScript()))
|
||||
.addAction("_action", ActionBuilders.loggingAction("some logging"))
|
||||
).get();
|
||||
}
|
||||
|
||||
|
@ -119,13 +129,13 @@ public class WatchStatsTests extends AbstractWatcherIntegrationTestCase {
|
|||
logger.info("Sleeping done, checking stats response");
|
||||
|
||||
WatcherStatsResponse response = watcherClient().prepareWatcherStats().setIncludeQueuedWatches(true).get();
|
||||
assertThat(response.getWatcherState(), equalTo(WatcherState.STARTED));
|
||||
boolean watcherStarted = response.getNodes().stream().allMatch(node -> node.getWatcherState() == WatcherState.STARTED);
|
||||
assertThat(watcherStarted, is(true));
|
||||
assertThat(response.getWatchesCount(), equalTo(5L));
|
||||
assertThat(response.getSnapshots(), nullValue());
|
||||
assertThat(response.getQueuedWatches(), notNullValue());
|
||||
assertThat(response.getQueuedWatches().size(), greaterThanOrEqualTo(5));
|
||||
assertThat(getSnapshots(response), hasSize(0));
|
||||
assertThat(getQueuedWatches(response), hasSize(greaterThanOrEqualTo(5)));
|
||||
DateTime previous = null;
|
||||
for (QueuedWatch queuedWatch : response.getQueuedWatches()) {
|
||||
for (QueuedWatch queuedWatch : getQueuedWatches(response)) {
|
||||
assertThat(queuedWatch.watchId(),
|
||||
anyOf(equalTo("_id0"), equalTo("_id1"), equalTo("_id2"), equalTo("_id3"), equalTo("_id4")));
|
||||
if (previous != null) {
|
||||
|
@ -137,6 +147,22 @@ public class WatchStatsTests extends AbstractWatcherIntegrationTestCase {
|
|||
logger.info("Pending watches test finished, now counting down latches");
|
||||
}
|
||||
|
||||
private List<WatchExecutionSnapshot> getSnapshots(WatcherStatsResponse response) {
|
||||
List<WatchExecutionSnapshot> snapshots = new ArrayList<>();
|
||||
response.getNodes().stream()
|
||||
.filter(node -> node.getSnapshots() != null)
|
||||
.forEach(node -> snapshots.addAll(node.getSnapshots()));
|
||||
return snapshots;
|
||||
}
|
||||
|
||||
private List<QueuedWatch> getQueuedWatches(WatcherStatsResponse response) {
|
||||
final List<QueuedWatch> queuedWatches = new ArrayList<>();
|
||||
response.getNodes().stream()
|
||||
.filter(node -> node.getQueuedWatches() != null)
|
||||
.forEach(node -> queuedWatches.addAll(node.getQueuedWatches()));
|
||||
return queuedWatches;
|
||||
}
|
||||
|
||||
private LatchScriptEngine getLatchScriptEngine() {
|
||||
return internalCluster().getInstance(LatchScriptEngine.LatchScriptPlugin.class).getScriptEngineService();
|
||||
}
|
||||
|
|
|
@ -7,18 +7,13 @@ package org.elasticsearch.xpack.watcher.transport.action.stats;
|
|||
|
||||
import org.elasticsearch.common.unit.TimeValue;
|
||||
import org.elasticsearch.test.ESIntegTestCase.ClusterScope;
|
||||
import org.elasticsearch.test.junit.annotations.TestLogging;
|
||||
import org.elasticsearch.xpack.watcher.WatcherState;
|
||||
import org.elasticsearch.xpack.watcher.client.WatcherClient;
|
||||
import org.elasticsearch.xpack.watcher.condition.CompareCondition;
|
||||
import org.elasticsearch.xpack.watcher.support.search.WatcherSearchTemplateRequest;
|
||||
import org.elasticsearch.xpack.watcher.test.AbstractWatcherIntegrationTestCase;
|
||||
import org.elasticsearch.xpack.watcher.transport.actions.delete.DeleteWatchResponse;
|
||||
import org.elasticsearch.xpack.watcher.transport.actions.stats.WatcherStatsRequest;
|
||||
import org.elasticsearch.xpack.watcher.transport.actions.stats.WatcherStatsResponse;
|
||||
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import static org.elasticsearch.index.query.QueryBuilders.termQuery;
|
||||
import static org.elasticsearch.search.builder.SearchSourceBuilder.searchSource;
|
||||
import static org.elasticsearch.test.ESIntegTestCase.Scope.TEST;
|
||||
|
@ -29,29 +24,29 @@ import static org.elasticsearch.xpack.watcher.trigger.TriggerBuilders.schedule;
|
|||
import static org.elasticsearch.xpack.watcher.trigger.schedule.Schedules.cron;
|
||||
import static org.hamcrest.Matchers.greaterThan;
|
||||
import static org.hamcrest.Matchers.is;
|
||||
import static org.hamcrest.core.IsEqual.equalTo;
|
||||
|
||||
@ClusterScope(scope = TEST, numClientNodes = 0, transportClientRatio = 0, randomDynamicTemplates = false)
|
||||
@TestLogging("org.elasticsearch.watcher:TRACE")
|
||||
public class WatcherStatsTests extends AbstractWatcherIntegrationTestCase {
|
||||
public void testStartedStats() throws Exception {
|
||||
WatcherStatsRequest watcherStatsRequest = watcherClient().prepareWatcherStats().request();
|
||||
WatcherStatsResponse response = watcherClient().watcherStats(watcherStatsRequest).actionGet();
|
||||
|
||||
assertThat(response.getWatcherState(), is(WatcherState.STARTED));
|
||||
assertThat(response.getThreadPoolQueueSize(), is(0L));
|
||||
@Override
|
||||
protected boolean timeWarped() {
|
||||
return true;
|
||||
}
|
||||
|
||||
public void testStartedStats() throws Exception {
|
||||
WatcherStatsResponse response = watcherClient().prepareWatcherStats().get();
|
||||
|
||||
assertThat(response.getWatchesCount(), is(0L));
|
||||
assertThat(response.getThreadPoolMaxSize(), is(timeWarped() ? 1L : 0L));
|
||||
response.getNodes().forEach(nodeResponse -> {
|
||||
assertThat(nodeResponse.getThreadPoolQueueSize(), is(0L));
|
||||
assertThat(nodeResponse.getThreadPoolMaxSize(), is(timeWarped() ? 1L : 0L));
|
||||
});
|
||||
|
||||
boolean isWatcherStarted = response.getNodes().stream().allMatch(node -> node.getWatcherState() == WatcherState.STARTED);
|
||||
assertThat("expected watcher to be started on all nodes", isWatcherStarted, is(true));
|
||||
}
|
||||
|
||||
public void testWatchCountStats() throws Exception {
|
||||
WatcherClient watcherClient = watcherClient();
|
||||
|
||||
WatcherStatsRequest watcherStatsRequest = watcherClient.prepareWatcherStats().request();
|
||||
WatcherStatsResponse response = watcherClient.watcherStats(watcherStatsRequest).actionGet();
|
||||
|
||||
assertThat(response.getWatcherState(), equalTo(WatcherState.STARTED));
|
||||
|
||||
WatcherSearchTemplateRequest request = templateRequest(searchSource().query(termQuery("field", "value")), "idx");
|
||||
watcherClient().preparePutWatch("_name")
|
||||
.setSource(watchBuilder()
|
||||
|
@ -61,20 +56,14 @@ public class WatcherStatsTests extends AbstractWatcherIntegrationTestCase {
|
|||
)
|
||||
.get();
|
||||
|
||||
if (timeWarped()) {
|
||||
timeWarp().scheduler().trigger("_name", 30, TimeValue.timeValueSeconds(1));
|
||||
} else {
|
||||
//Wait a little until we should have queued an action
|
||||
Thread.sleep(TimeUnit.SECONDS.toMillis(5));
|
||||
}
|
||||
timeWarp().trigger("_name", 30, TimeValue.timeValueSeconds(1));
|
||||
|
||||
response = watcherClient().watcherStats(watcherStatsRequest).actionGet();
|
||||
WatcherStatsResponse response = watcherClient().prepareWatcherStats().get();
|
||||
|
||||
assertThat(response.getWatcherState(), is(WatcherState.STARTED));
|
||||
assertThat(response.getWatchesCount(), is(1L));
|
||||
assertThat(response.getThreadPoolMaxSize(), greaterThan(0L));
|
||||
response.getNodes().forEach(nodeResponse -> assertThat(nodeResponse.getThreadPoolMaxSize(), greaterThan(0L)));
|
||||
|
||||
DeleteWatchResponse deleteWatchResponse = watcherClient.prepareDeleteWatch("_name").get();
|
||||
DeleteWatchResponse deleteWatchResponse = watcherClient().prepareDeleteWatch("_name").get();
|
||||
assertThat(deleteWatchResponse.isFound(), is(true));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -27,7 +27,7 @@ import java.util.concurrent.ConcurrentMap;
|
|||
|
||||
/**
|
||||
* A mock scheduler to help with unit testing. Provide {@link ScheduleTriggerEngineMock#trigger} method to manually trigger
|
||||
* jobs.
|
||||
* jobCount.
|
||||
*/
|
||||
public class ScheduleTriggerEngineMock extends ScheduleTriggerEngine {
|
||||
|
||||
|
@ -56,13 +56,25 @@ public class ScheduleTriggerEngineMock extends ScheduleTriggerEngine {
|
|||
|
||||
@Override
|
||||
public void stop() {
|
||||
watches.clear();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void add(Watch watch) {
|
||||
logger.debug("adding watch [{}]", watch.id());
|
||||
watches.put(watch.id(), watch);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void pauseExecution() {
|
||||
watches.clear();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getJobCount() {
|
||||
return watches.size();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean remove(String jobId) {
|
||||
return watches.remove(jobId) != null;
|
||||
|
@ -77,9 +89,14 @@ public class ScheduleTriggerEngineMock extends ScheduleTriggerEngine {
|
|||
}
|
||||
|
||||
public void trigger(String jobName, int times, TimeValue interval) {
|
||||
if (watches.containsKey(jobName) == false) {
|
||||
logger.trace("not executing job [{}], not found", jobName);
|
||||
return;
|
||||
}
|
||||
|
||||
for (int i = 0; i < times; i++) {
|
||||
DateTime now = new DateTime(clock.millis());
|
||||
logger.debug("firing [{}] at [{}]", jobName, now);
|
||||
logger.debug("firing watch [{}] at [{}]", jobName, now);
|
||||
ScheduleTriggerEvent event = new ScheduleTriggerEvent(jobName, now, now);
|
||||
consumers.forEach(consumer -> consumer.accept(Collections.singletonList(event)));
|
||||
if (interval != null) {
|
||||
|
|
|
@ -38,7 +38,7 @@ public class WatchStatusIntegrationTests extends AbstractWatcherIntegrationTestC
|
|||
.condition(NeverCondition.INSTANCE)
|
||||
.addAction("_logger", loggingAction("logged text")))
|
||||
.get();
|
||||
timeWarp().scheduler().trigger("_name");
|
||||
timeWarp().trigger("_name");
|
||||
|
||||
GetWatchResponse getWatchResponse = watcherClient.prepareGetWatch().setId("_name").get();
|
||||
assertThat(getWatchResponse.isFound(), is(true));
|
||||
|
|
|
@ -543,6 +543,15 @@ public class WatchTests extends ESTestCase {
|
|||
public void add(Watch watch) {
|
||||
}
|
||||
|
||||
@Override
|
||||
public void pauseExecution() {
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getJobCount() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean remove(String jobId) {
|
||||
return false;
|
||||
|
|
|
@ -59,6 +59,7 @@ teardown:
|
|||
- match: { watch_record.trigger_event.manual.schedule.scheduled_time: "2000-12-12T12:12:12.120Z" }
|
||||
- match: { watch_record.state: "executed" }
|
||||
- match: { watch_record._status.state.active: true }
|
||||
- is_true: watch_record.node
|
||||
- match: { watch_record._status.actions.indexme.ack.state: "ackable" }
|
||||
|
||||
---
|
||||
|
|
|
@ -2,5 +2,5 @@
|
|||
"Test watcher stats output":
|
||||
|
||||
- do: {xpack.watcher.stats: {}}
|
||||
- match: { "watcher_state": "started" }
|
||||
- match: { "manually_stopped": false }
|
||||
- match: { "stats.0.watcher_state": "started" }
|
||||
|
|
|
@ -0,0 +1,289 @@
|
|||
/*
|
||||
* 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.HttpHost;
|
||||
import org.apache.http.entity.ContentType;
|
||||
import org.apache.http.entity.StringEntity;
|
||||
import org.apache.http.util.EntityUtils;
|
||||
import org.elasticsearch.Version;
|
||||
import org.elasticsearch.client.Response;
|
||||
import org.elasticsearch.client.RestClient;
|
||||
import org.elasticsearch.common.CheckedConsumer;
|
||||
import org.elasticsearch.common.bytes.BytesReference;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.util.concurrent.ThreadContext;
|
||||
import org.elasticsearch.common.xcontent.XContentType;
|
||||
import org.elasticsearch.test.rest.ESRestTestCase;
|
||||
import org.elasticsearch.test.rest.yaml.ObjectPath;
|
||||
import org.elasticsearch.xpack.watcher.condition.AlwaysCondition;
|
||||
import org.junit.Before;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.Base64;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static org.elasticsearch.xpack.security.SecurityLifecycleService.SECURITY_TEMPLATE_NAME;
|
||||
import static org.elasticsearch.xpack.watcher.actions.ActionBuilders.loggingAction;
|
||||
import static org.elasticsearch.xpack.watcher.client.WatchSourceBuilders.watchBuilder;
|
||||
import static org.elasticsearch.xpack.watcher.input.InputBuilders.simpleInput;
|
||||
import static org.elasticsearch.xpack.watcher.trigger.TriggerBuilders.schedule;
|
||||
import static org.elasticsearch.xpack.watcher.trigger.schedule.Schedules.interval;
|
||||
import static org.hamcrest.Matchers.anyOf;
|
||||
import static org.hamcrest.Matchers.equalTo;
|
||||
import static org.hamcrest.Matchers.greaterThanOrEqualTo;
|
||||
|
||||
public class WatchBackwardsCompatibilityIT extends ESRestTestCase {
|
||||
|
||||
@Before
|
||||
public void waitForSecuritySetup() throws Exception {
|
||||
String masterNode = null;
|
||||
String catNodesResponse = EntityUtils.toString(
|
||||
client().performRequest("GET", "/_cat/nodes?h=id,master").getEntity(),
|
||||
StandardCharsets.UTF_8
|
||||
);
|
||||
for (String line : catNodesResponse.split("\n")) {
|
||||
int indexOfStar = line.indexOf('*'); // * in the node's output denotes it is master
|
||||
if (indexOfStar != -1) {
|
||||
masterNode = line.substring(0, indexOfStar).trim();
|
||||
break;
|
||||
}
|
||||
}
|
||||
assertNotNull(masterNode);
|
||||
final String masterNodeId = masterNode;
|
||||
|
||||
assertBusy(() -> {
|
||||
try {
|
||||
Response nodeDetailsResponse = client().performRequest("GET", "/_nodes");
|
||||
ObjectPath path = ObjectPath.createFromResponse(nodeDetailsResponse);
|
||||
Map<String, Object> nodes = path.evaluate("nodes");
|
||||
assertThat(nodes.size(), greaterThanOrEqualTo(2));
|
||||
String masterVersion = null;
|
||||
for (String key : nodes.keySet()) {
|
||||
// get the ES version number master is on
|
||||
if (key.startsWith(masterNodeId)) {
|
||||
masterVersion = path.evaluate("nodes." + key + ".version");
|
||||
break;
|
||||
}
|
||||
}
|
||||
assertNotNull(masterVersion);
|
||||
final String masterTemplateVersion = masterVersion;
|
||||
|
||||
Response response = client().performRequest("GET", "/_cluster/state/metadata");
|
||||
ObjectPath objectPath = ObjectPath.createFromResponse(response);
|
||||
final String mappingsPath = "metadata.templates." + SECURITY_TEMPLATE_NAME + "" +
|
||||
".mappings";
|
||||
Map<String, Object> mappings = objectPath.evaluate(mappingsPath);
|
||||
assertNotNull(mappings);
|
||||
assertThat(mappings.size(), greaterThanOrEqualTo(1));
|
||||
for (String key : mappings.keySet()) {
|
||||
String templateVersion = objectPath.evaluate(mappingsPath + "." + key + "" +
|
||||
"._meta.security-version");
|
||||
assertEquals(masterTemplateVersion, templateVersion);
|
||||
}
|
||||
} catch (Exception e) {
|
||||
throw new AssertionError("failed to get cluster state", e);
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
private Nodes nodes;
|
||||
|
||||
@Before
|
||||
public void ensureNewNodesAreInCluster() throws IOException {
|
||||
nodes = buildNodeAndVersions();
|
||||
assumeFalse("new nodes is empty", nodes.getNewNodes().isEmpty());
|
||||
|
||||
}
|
||||
|
||||
public void testWatcherStats() throws Exception {
|
||||
executeAgainstAllNodes(client ->
|
||||
assertOK(client.performRequest("GET", "/_xpack/watcher/stats"))
|
||||
);
|
||||
}
|
||||
|
||||
public void testWatcherRestart() throws Exception {
|
||||
executeAgainstAllNodes(client -> {
|
||||
assertOK(client.performRequest("POST", "/_xpack/watcher/_stop"));
|
||||
assertOK(client.performRequest("POST", "/_xpack/watcher/_start"));
|
||||
});
|
||||
}
|
||||
|
||||
public void testWatchCrudApis() throws IOException {
|
||||
BytesReference bytesReference = watchBuilder()
|
||||
.trigger(schedule(interval("5m")))
|
||||
.input(simpleInput())
|
||||
.condition(AlwaysCondition.INSTANCE)
|
||||
.addAction("_action1", loggingAction("{{ctx.watch_id}}"))
|
||||
.buildAsBytes(XContentType.JSON);
|
||||
StringEntity entity = new StringEntity(bytesReference.utf8ToString(),
|
||||
ContentType.APPLICATION_JSON);
|
||||
|
||||
executeAgainstAllNodes(client -> {
|
||||
assertOK(client.performRequest("PUT", "/_xpack/watcher/watch/my-watch",
|
||||
Collections.emptyMap(), entity));
|
||||
|
||||
assertOK(client.performRequest("GET", "/_xpack/watcher/watch/my-watch"));
|
||||
|
||||
assertOK(client.performRequest("POST", "/_xpack/watcher/watch/my-watch/_execute"));
|
||||
|
||||
assertOK(client.performRequest("PUT", "/_xpack/watcher/watch/my-watch/_deactivate"));
|
||||
|
||||
assertOK(client.performRequest("PUT", "/_xpack/watcher/watch/my-watch/_activate"));
|
||||
});
|
||||
}
|
||||
|
||||
public void executeAgainstAllNodes(CheckedConsumer<RestClient, IOException> consumer)
|
||||
throws IOException {
|
||||
try (RestClient newClient = buildClient(restClientSettings(),
|
||||
nodes.getNewNodes().stream().map(Node::getPublishAddress)
|
||||
.toArray(HttpHost[]::new))) {
|
||||
consumer.accept(newClient);
|
||||
}
|
||||
|
||||
try (RestClient bwcClient = buildClient(restClientSettings(),
|
||||
nodes.getBWCNodes().stream().map(Node::getPublishAddress)
|
||||
.toArray(HttpHost[]::new))) {
|
||||
consumer.accept(bwcClient);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean preserveIndicesUponCompletion() {
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Settings restClientSettings() {
|
||||
String token = "Basic " + Base64.getEncoder()
|
||||
.encodeToString("elastic:changeme".getBytes(StandardCharsets.UTF_8));
|
||||
return Settings.builder()
|
||||
.put(ThreadContext.PREFIX + ".Authorization", token)
|
||||
.build();
|
||||
}
|
||||
|
||||
private void assertOK(Response response) {
|
||||
assertThat(response.getStatusLine().getStatusCode(), anyOf(equalTo(200), equalTo(201)));
|
||||
}
|
||||
|
||||
private Nodes buildNodeAndVersions() throws IOException {
|
||||
Response response = client().performRequest("GET", "_nodes");
|
||||
ObjectPath objectPath = ObjectPath.createFromResponse(response);
|
||||
Map<String, Object> nodesAsMap = objectPath.evaluate("nodes");
|
||||
Nodes nodes = new Nodes();
|
||||
for (String id : nodesAsMap.keySet()) {
|
||||
nodes.add(new Node(
|
||||
id,
|
||||
objectPath.evaluate("nodes." + id + ".name"),
|
||||
Version.fromString(objectPath.evaluate("nodes." + id + ".version")),
|
||||
HttpHost.create(objectPath.evaluate("nodes." + id + ".http.publish_address"))));
|
||||
}
|
||||
response = client().performRequest("GET", "_cluster/state");
|
||||
nodes.setMasterNodeId(ObjectPath.createFromResponse(response).evaluate("master_node"));
|
||||
return nodes;
|
||||
}
|
||||
|
||||
final class Nodes extends HashMap<String, Node> {
|
||||
|
||||
private String masterNodeId = null;
|
||||
|
||||
public Node getMaster() {
|
||||
return get(masterNodeId);
|
||||
}
|
||||
|
||||
public void setMasterNodeId(String id) {
|
||||
if (get(id) == null) {
|
||||
throw new IllegalArgumentException("node with id [" + id + "] not found. got:" +
|
||||
toString());
|
||||
}
|
||||
masterNodeId = id;
|
||||
}
|
||||
|
||||
public void add(Node node) {
|
||||
put(node.getId(), node);
|
||||
}
|
||||
|
||||
public List<Node> getNewNodes() {
|
||||
Version bwcVersion = getBWCVersion();
|
||||
return values().stream().filter(n -> n.getVersion().after(bwcVersion))
|
||||
.collect(Collectors.toList());
|
||||
}
|
||||
|
||||
public List<Node> getBWCNodes() {
|
||||
Version bwcVersion = getBWCVersion();
|
||||
return values().stream().filter(n -> n.getVersion().equals(bwcVersion))
|
||||
.collect(Collectors.toList());
|
||||
}
|
||||
|
||||
public Version getBWCVersion() {
|
||||
if (isEmpty()) {
|
||||
throw new IllegalStateException("no nodes available");
|
||||
}
|
||||
return Version.fromId(values().stream().map(node -> node.getVersion().id)
|
||||
.min(Integer::compareTo).get());
|
||||
}
|
||||
|
||||
public Node getSafe(String id) {
|
||||
Node node = get(id);
|
||||
if (node == null) {
|
||||
throw new IllegalArgumentException("node with id [" + id + "] not found");
|
||||
}
|
||||
return node;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "Nodes{" +
|
||||
"masterNodeId='" + masterNodeId + "'\n" +
|
||||
values().stream().map(Node::toString).collect(Collectors.joining("\n")) +
|
||||
'}';
|
||||
}
|
||||
}
|
||||
|
||||
final class Node {
|
||||
private final String id;
|
||||
private final String nodeName;
|
||||
private final Version version;
|
||||
private final HttpHost publishAddress;
|
||||
|
||||
Node(String id, String nodeName, Version version, HttpHost publishAddress) {
|
||||
this.id = id;
|
||||
this.nodeName = nodeName;
|
||||
this.version = version;
|
||||
this.publishAddress = publishAddress;
|
||||
}
|
||||
|
||||
public String getId() {
|
||||
return id;
|
||||
}
|
||||
|
||||
public String getNodeName() {
|
||||
return nodeName;
|
||||
}
|
||||
|
||||
public HttpHost getPublishAddress() {
|
||||
return publishAddress;
|
||||
}
|
||||
|
||||
public Version getVersion() {
|
||||
return version;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "Node{" +
|
||||
"id='" + id + '\'' +
|
||||
", nodeName='" + nodeName + '\'' +
|
||||
", version=" + version +
|
||||
'}';
|
||||
}
|
||||
}
|
||||
}
|
|
@ -1,39 +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.smoketest;
|
||||
|
||||
import com.carrotsearch.randomizedtesting.annotations.Name;
|
||||
import com.carrotsearch.randomizedtesting.annotations.ParametersFactory;
|
||||
|
||||
import org.elasticsearch.test.rest.yaml.ClientYamlTestCandidate;
|
||||
import org.elasticsearch.test.rest.yaml.ESClientYamlSuiteTestCase;
|
||||
import org.junit.After;
|
||||
import org.junit.Before;
|
||||
|
||||
import static java.util.Collections.emptyList;
|
||||
import static java.util.Collections.emptyMap;
|
||||
|
||||
public abstract class SmokeTestWatchesWithMustacheClientYamlTestSuiteTestCase extends ESClientYamlSuiteTestCase {
|
||||
|
||||
public SmokeTestWatchesWithMustacheClientYamlTestSuiteTestCase(@Name("yaml") ClientYamlTestCandidate testCandidate) {
|
||||
super(testCandidate);
|
||||
}
|
||||
|
||||
@ParametersFactory
|
||||
public static Iterable<Object[]> parameters() throws Exception {
|
||||
return ESClientYamlSuiteTestCase.createParameters();
|
||||
}
|
||||
|
||||
@Before
|
||||
public void startWatcher() throws Exception {
|
||||
getAdminExecutionContext().callApi("xpack.watcher.start", emptyMap(), emptyList(), emptyMap());
|
||||
}
|
||||
|
||||
@After
|
||||
public void stopWatcher() throws Exception {
|
||||
getAdminExecutionContext().callApi("xpack.watcher.stop", emptyMap(), emptyList(), emptyMap());
|
||||
}
|
||||
}
|
|
@ -7,12 +7,20 @@ package org.elasticsearch.smoketest;
|
|||
|
||||
import com.carrotsearch.randomizedtesting.annotations.Name;
|
||||
import com.carrotsearch.randomizedtesting.annotations.ParametersFactory;
|
||||
|
||||
import org.elasticsearch.test.rest.yaml.ClientYamlTestCandidate;
|
||||
import org.elasticsearch.test.rest.yaml.ClientYamlTestResponse;
|
||||
import org.elasticsearch.test.rest.yaml.ESClientYamlSuiteTestCase;
|
||||
import org.junit.After;
|
||||
import org.junit.Before;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import static java.util.Collections.emptyList;
|
||||
import static java.util.Collections.emptyMap;
|
||||
import static org.hamcrest.Matchers.is;
|
||||
|
||||
/** Runs rest tests against external cluster */
|
||||
public class WatcherWithMustacheIT extends SmokeTestWatchesWithMustacheClientYamlTestSuiteTestCase {
|
||||
public class WatcherWithMustacheIT extends ESClientYamlSuiteTestCase {
|
||||
|
||||
public WatcherWithMustacheIT(@Name("yaml") ClientYamlTestCandidate testCandidate) {
|
||||
super(testCandidate);
|
||||
|
@ -23,4 +31,33 @@ public class WatcherWithMustacheIT extends SmokeTestWatchesWithMustacheClientYam
|
|||
return ESClientYamlSuiteTestCase.createParameters();
|
||||
}
|
||||
|
||||
@Before
|
||||
public void startWatcher() throws Exception {
|
||||
assertBusy(() -> {
|
||||
try {
|
||||
getAdminExecutionContext().callApi("xpack.watcher.start", emptyMap(), emptyList(), emptyMap());
|
||||
ClientYamlTestResponse response =
|
||||
getAdminExecutionContext().callApi("xpack.watcher.stats", emptyMap(), emptyList(), emptyMap());
|
||||
String state = (String) response.evaluate("stats.0.watcher_state");
|
||||
assertThat(state, is("started"));
|
||||
} catch (IOException e) {
|
||||
throw new AssertionError(e);
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
@After
|
||||
public void stopWatcher() throws Exception {
|
||||
assertBusy(() -> {
|
||||
try {
|
||||
getAdminExecutionContext().callApi("xpack.watcher.stop", emptyMap(), emptyList(), emptyMap());
|
||||
ClientYamlTestResponse response =
|
||||
getAdminExecutionContext().callApi("xpack.watcher.stats", emptyMap(), emptyList(), emptyMap());
|
||||
String state = (String) response.evaluate("stats.0.watcher_state");
|
||||
assertThat(state, is("stopped"));
|
||||
} catch (IOException e) {
|
||||
throw new AssertionError(e);
|
||||
}
|
||||
});
|
||||
}
|
||||
}
|
||||
|
|
|
@ -7,8 +7,8 @@
|
|||
|
||||
- do:
|
||||
xpack.watcher.stats: {}
|
||||
- match: { "watcher_state": "started" }
|
||||
- match: { "watch_count": 0 }
|
||||
- match: { "stats.0.watcher_state": "started" }
|
||||
- match: { "stats.0.watch_count": 0 }
|
||||
|
||||
- do:
|
||||
xpack.watcher.put_watch:
|
||||
|
@ -17,7 +17,7 @@
|
|||
{
|
||||
"trigger": {
|
||||
"schedule": {
|
||||
"interval": "1s"
|
||||
"interval": "5m"
|
||||
}
|
||||
},
|
||||
"input": {
|
||||
|
@ -92,7 +92,7 @@
|
|||
|
||||
- do:
|
||||
xpack.watcher.stats: {}
|
||||
- match: { "watch_count": 1 }
|
||||
- match: { "stats.0.watch_count": 1 }
|
||||
|
||||
- do:
|
||||
xpack.watcher.execute_watch:
|
||||
|
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue