Răsfoiți Sursa

Added ingest info to node info API, which contains a list of available processors.

Internally the put pipeline API uses this information in node info API to validate if all specified processors in a pipeline exist on all nodes in the cluster.
Martijn van Groningen 9 ani în urmă
părinte
comite
82d01e4315
26 a modificat fișierele cu 649 adăugiri și 28 ștergeri
  1. 22 2
      core/src/main/java/org/elasticsearch/action/admin/cluster/node/info/NodeInfo.java
  2. 21 0
      core/src/main/java/org/elasticsearch/action/admin/cluster/node/info/NodesInfoRequest.java
  3. 8 0
      core/src/main/java/org/elasticsearch/action/admin/cluster/node/info/NodesInfoRequestBuilder.java
  4. 3 0
      core/src/main/java/org/elasticsearch/action/admin/cluster/node/info/NodesInfoResponse.java
  5. 2 2
      core/src/main/java/org/elasticsearch/action/admin/cluster/node/info/TransportNodesInfoAction.java
  6. 1 1
      core/src/main/java/org/elasticsearch/action/admin/cluster/stats/TransportClusterStatsAction.java
  7. 35 2
      core/src/main/java/org/elasticsearch/action/ingest/PutPipelineTransportAction.java
  8. 15 0
      core/src/main/java/org/elasticsearch/ingest/IngestService.java
  9. 28 10
      core/src/main/java/org/elasticsearch/ingest/PipelineStore.java
  10. 1 0
      core/src/main/java/org/elasticsearch/ingest/ProcessorsRegistry.java
  11. 21 0
      core/src/main/java/org/elasticsearch/ingest/core/CompoundProcessor.java
  12. 100 0
      core/src/main/java/org/elasticsearch/ingest/core/IngestInfo.java
  13. 8 0
      core/src/main/java/org/elasticsearch/ingest/core/Pipeline.java
  14. 81 0
      core/src/main/java/org/elasticsearch/ingest/core/ProcessorInfo.java
  15. 5 3
      core/src/main/java/org/elasticsearch/node/service/NodeService.java
  16. 2 1
      core/src/main/java/org/elasticsearch/rest/action/admin/cluster/node/info/RestNodesInfoAction.java
  17. 112 0
      core/src/test/java/org/elasticsearch/ingest/IngestProcessorNotInstalledOnAllNodesIT.java
  18. 43 0
      core/src/test/java/org/elasticsearch/ingest/PipelineStoreTests.java
  19. 12 0
      core/src/test/java/org/elasticsearch/ingest/core/PipelineFactoryTests.java
  20. 4 1
      core/src/test/java/org/elasticsearch/nodesinfo/NodeInfoStreamingTests.java
  21. 69 1
      docs/reference/cluster/nodes-info.asciidoc
  22. 14 0
      docs/reference/ingest/ingest-node.asciidoc
  23. 8 2
      plugins/ingest-attachment/src/test/resources/rest-api-spec/test/ingest_attachment/10_basic.yaml
  24. 8 2
      plugins/ingest-geoip/src/test/resources/rest-api-spec/test/ingest_geoip/10_basic.yaml
  25. 1 1
      rest-api-spec/src/main/resources/rest-api-spec/api/nodes.info.json
  26. 25 0
      rest-api-spec/src/main/resources/rest-api-spec/test/ingest/10_crud.yaml

+ 22 - 2
core/src/main/java/org/elasticsearch/action/admin/cluster/node/info/NodeInfo.java

@@ -28,6 +28,7 @@ import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.http.HttpInfo;
+import org.elasticsearch.ingest.core.IngestInfo;
 import org.elasticsearch.monitor.jvm.JvmInfo;
 import org.elasticsearch.monitor.os.OsInfo;
 import org.elasticsearch.monitor.process.ProcessInfo;
@@ -74,12 +75,15 @@ public class NodeInfo extends BaseNodeResponse {
     @Nullable
     private PluginsAndModules plugins;
 
-    NodeInfo() {
+    @Nullable
+    private IngestInfo ingest;
+
+    public NodeInfo() {
     }
 
     public NodeInfo(Version version, Build build, DiscoveryNode node, @Nullable Map<String, String> serviceAttributes, @Nullable Settings settings,
                     @Nullable OsInfo os, @Nullable ProcessInfo process, @Nullable JvmInfo jvm, @Nullable ThreadPoolInfo threadPool,
-                    @Nullable TransportInfo transport, @Nullable HttpInfo http, @Nullable PluginsAndModules plugins) {
+                    @Nullable TransportInfo transport, @Nullable HttpInfo http, @Nullable PluginsAndModules plugins, @Nullable IngestInfo ingest) {
         super(node);
         this.version = version;
         this.build = build;
@@ -92,6 +96,7 @@ public class NodeInfo extends BaseNodeResponse {
         this.transport = transport;
         this.http = http;
         this.plugins = plugins;
+        this.ingest = ingest;
     }
 
     /**
@@ -176,6 +181,11 @@ public class NodeInfo extends BaseNodeResponse {
         return this.plugins;
     }
 
+    @Nullable
+    public IngestInfo getIngest() {
+        return ingest;
+    }
+
     public static NodeInfo readNodeInfo(StreamInput in) throws IOException {
         NodeInfo nodeInfo = new NodeInfo();
         nodeInfo.readFrom(in);
@@ -220,6 +230,10 @@ public class NodeInfo extends BaseNodeResponse {
             plugins = new PluginsAndModules();
             plugins.readFrom(in);
         }
+        if (in.readBoolean()) {
+            ingest = new IngestInfo();
+            ingest.readFrom(in);
+        }
     }
 
     @Override
@@ -285,5 +299,11 @@ public class NodeInfo extends BaseNodeResponse {
             out.writeBoolean(true);
             plugins.writeTo(out);
         }
+        if (ingest == null) {
+            out.writeBoolean(false);
+        } else {
+            out.writeBoolean(true);
+            ingest.writeTo(out);
+        }
     }
 }

+ 21 - 0
core/src/main/java/org/elasticsearch/action/admin/cluster/node/info/NodesInfoRequest.java

@@ -38,6 +38,7 @@ public class NodesInfoRequest extends BaseNodesRequest<NodesInfoRequest> {
     private boolean transport = true;
     private boolean http = true;
     private boolean plugins = true;
+    private boolean ingest = true;
 
     public NodesInfoRequest() {
     }
@@ -62,6 +63,7 @@ public class NodesInfoRequest extends BaseNodesRequest<NodesInfoRequest> {
         transport = false;
         http = false;
         plugins = false;
+        ingest = false;
         return this;
     }
 
@@ -77,6 +79,7 @@ public class NodesInfoRequest extends BaseNodesRequest<NodesInfoRequest> {
         transport = true;
         http = true;
         plugins = true;
+        ingest = true;
         return this;
     }
 
@@ -202,6 +205,22 @@ public class NodesInfoRequest extends BaseNodesRequest<NodesInfoRequest> {
         return plugins;
     }
 
+    /**
+     * Should information about ingest be returned
+     * @param ingest true if you want info
+     */
+    public NodesInfoRequest ingest(boolean ingest) {
+        this.ingest = ingest;
+        return this;
+    }
+
+    /**
+     * @return true if information about ingest is requested
+     */
+    public boolean ingest() {
+        return ingest;
+    }
+
     @Override
     public void readFrom(StreamInput in) throws IOException {
         super.readFrom(in);
@@ -213,6 +232,7 @@ public class NodesInfoRequest extends BaseNodesRequest<NodesInfoRequest> {
         transport = in.readBoolean();
         http = in.readBoolean();
         plugins = in.readBoolean();
+        ingest = in.readBoolean();
     }
 
     @Override
@@ -226,5 +246,6 @@ public class NodesInfoRequest extends BaseNodesRequest<NodesInfoRequest> {
         out.writeBoolean(transport);
         out.writeBoolean(http);
         out.writeBoolean(plugins);
+        out.writeBoolean(ingest);
     }
 }

+ 8 - 0
core/src/main/java/org/elasticsearch/action/admin/cluster/node/info/NodesInfoRequestBuilder.java

@@ -110,4 +110,12 @@ public class NodesInfoRequestBuilder extends NodesOperationRequestBuilder<NodesI
         request().plugins(plugins);
         return this;
     }
+
+    /**
+     * Should the node ingest info be returned.
+     */
+    public NodesInfoRequestBuilder setIngest(boolean ingest) {
+        request().ingest(ingest);
+        return this;
+    }
 }

+ 3 - 0
core/src/main/java/org/elasticsearch/action/admin/cluster/node/info/NodesInfoResponse.java

@@ -121,6 +121,9 @@ public class NodesInfoResponse extends BaseNodesResponse<NodeInfo> implements To
             if (nodeInfo.getPlugins() != null) {
                 nodeInfo.getPlugins().toXContent(builder, params);
             }
+            if (nodeInfo.getIngest() != null) {
+                nodeInfo.getIngest().toXContent(builder, params);
+            }
 
             builder.endObject();
         }

+ 2 - 2
core/src/main/java/org/elasticsearch/action/admin/cluster/node/info/TransportNodesInfoAction.java

@@ -80,7 +80,7 @@ public class TransportNodesInfoAction extends TransportNodesAction<NodesInfoRequ
     protected NodeInfo nodeOperation(NodeInfoRequest nodeRequest) {
         NodesInfoRequest request = nodeRequest.request;
         return nodeService.info(request.settings(), request.os(), request.process(), request.jvm(), request.threadPool(),
-                request.transport(), request.http(), request.plugins());
+                request.transport(), request.http(), request.plugins(), request.ingest());
     }
 
     @Override
@@ -95,7 +95,7 @@ public class TransportNodesInfoAction extends TransportNodesAction<NodesInfoRequ
         public NodeInfoRequest() {
         }
 
-        NodeInfoRequest(String nodeId, NodesInfoRequest request) {
+        public NodeInfoRequest(String nodeId, NodesInfoRequest request) {
             super(nodeId);
             this.request = request;
         }

+ 1 - 1
core/src/main/java/org/elasticsearch/action/admin/cluster/stats/TransportClusterStatsAction.java

@@ -98,7 +98,7 @@ public class TransportClusterStatsAction extends TransportNodesAction<ClusterSta
 
     @Override
     protected ClusterStatsNodeResponse nodeOperation(ClusterStatsNodeRequest nodeRequest) {
-        NodeInfo nodeInfo = nodeService.info(false, true, false, true, false, true, false, true);
+        NodeInfo nodeInfo = nodeService.info(false, true, false, true, false, true, false, true, false);
         NodeStats nodeStats = nodeService.stats(CommonStatsFlags.NONE, false, true, true, false, true, false, false, false, false, false);
         List<ShardStats> shardsStats = new ArrayList<>();
         for (IndexService indexService : indicesService) {

+ 35 - 2
core/src/main/java/org/elasticsearch/action/ingest/PutPipelineTransportAction.java

@@ -20,6 +20,10 @@
 package org.elasticsearch.action.ingest;
 
 import org.elasticsearch.action.ActionListener;
+import org.elasticsearch.action.admin.cluster.node.info.NodeInfo;
+import org.elasticsearch.action.admin.cluster.node.info.NodesInfoRequest;
+import org.elasticsearch.action.admin.cluster.node.info.NodesInfoResponse;
+import org.elasticsearch.action.admin.cluster.node.info.TransportNodesInfoAction;
 import org.elasticsearch.action.support.ActionFilters;
 import org.elasticsearch.action.support.master.TransportMasterNodeAction;
 import org.elasticsearch.cluster.ClusterService;
@@ -27,24 +31,32 @@ 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.node.DiscoveryNode;
 import org.elasticsearch.common.inject.Inject;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.ingest.PipelineStore;
+import org.elasticsearch.ingest.core.IngestInfo;
 import org.elasticsearch.node.service.NodeService;
 import org.elasticsearch.threadpool.ThreadPool;
 import org.elasticsearch.transport.TransportService;
 
+import java.util.HashMap;
+import java.util.Map;
+
 public class PutPipelineTransportAction extends TransportMasterNodeAction<PutPipelineRequest, WritePipelineResponse> {
 
     private final PipelineStore pipelineStore;
     private final ClusterService clusterService;
+    private final TransportNodesInfoAction nodesInfoAction;
 
     @Inject
     public PutPipelineTransportAction(Settings settings, ThreadPool threadPool, ClusterService clusterService,
                                       TransportService transportService, ActionFilters actionFilters,
-                                      IndexNameExpressionResolver indexNameExpressionResolver, NodeService nodeService) {
+                                      IndexNameExpressionResolver indexNameExpressionResolver, NodeService nodeService,
+                                      TransportNodesInfoAction nodesInfoAction) {
         super(settings, PutPipelineAction.NAME, transportService, clusterService, threadPool, actionFilters, indexNameExpressionResolver, PutPipelineRequest::new);
         this.clusterService = clusterService;
+        this.nodesInfoAction = nodesInfoAction;
         this.pipelineStore = nodeService.getIngestService().getPipelineStore();
     }
 
@@ -60,7 +72,28 @@ public class PutPipelineTransportAction extends TransportMasterNodeAction<PutPip
 
     @Override
     protected void masterOperation(PutPipelineRequest request, ClusterState state, ActionListener<WritePipelineResponse> listener) throws Exception {
-        pipelineStore.put(clusterService, request, listener);
+        NodesInfoRequest nodesInfoRequest = new NodesInfoRequest();
+        nodesInfoRequest.clear();
+        nodesInfoRequest.ingest(true);
+        nodesInfoAction.execute(nodesInfoRequest, new ActionListener<NodesInfoResponse>() {
+            @Override
+            public void onResponse(NodesInfoResponse nodeInfos) {
+                try {
+                    Map<DiscoveryNode, IngestInfo> ingestInfos = new HashMap<>();
+                    for (NodeInfo nodeInfo : nodeInfos) {
+                        ingestInfos.put(nodeInfo.getNode(), nodeInfo.getIngest());
+                    }
+                    pipelineStore.put(clusterService, ingestInfos, request, listener);
+                } catch (Exception e) {
+                    onFailure(e);
+                }
+            }
+
+            @Override
+            public void onFailure(Throwable e) {
+                listener.onFailure(e);
+            }
+        });
     }
 
     @Override

+ 15 - 0
core/src/main/java/org/elasticsearch/ingest/IngestService.java

@@ -20,11 +20,17 @@
 package org.elasticsearch.ingest;
 
 import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.ingest.core.IngestInfo;
+import org.elasticsearch.ingest.core.Processor;
+import org.elasticsearch.ingest.core.ProcessorInfo;
 import org.elasticsearch.script.ScriptService;
 import org.elasticsearch.threadpool.ThreadPool;
 
 import java.io.Closeable;
 import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
 
 /**
  * Holder class for several ingest related services.
@@ -53,6 +59,15 @@ public class IngestService implements Closeable {
         pipelineStore.buildProcessorFactoryRegistry(processorsRegistryBuilder, scriptService);
     }
 
+    public IngestInfo info() {
+        Map<String, Processor.Factory> processorFactories = pipelineStore.getProcessorRegistry().getProcessorFactories();
+        List<ProcessorInfo> processorInfoList = new ArrayList<>(processorFactories.size());
+        for (Map.Entry<String, Processor.Factory> entry : processorFactories.entrySet()) {
+            processorInfoList.add(new ProcessorInfo(entry.getKey()));
+        }
+        return new IngestInfo(processorInfoList);
+    }
+
     @Override
     public void close() throws IOException {
         pipelineStore.close();

+ 28 - 10
core/src/main/java/org/elasticsearch/ingest/PipelineStore.java

@@ -20,6 +20,7 @@
 package org.elasticsearch.ingest;
 
 import org.elasticsearch.ElasticsearchParseException;
+import org.elasticsearch.ExceptionsHelper;
 import org.elasticsearch.ResourceNotFoundException;
 import org.elasticsearch.action.ActionListener;
 import org.elasticsearch.action.ingest.DeletePipelineRequest;
@@ -31,12 +32,15 @@ import org.elasticsearch.cluster.ClusterService;
 import org.elasticsearch.cluster.ClusterState;
 import org.elasticsearch.cluster.ClusterStateListener;
 import org.elasticsearch.cluster.metadata.MetaData;
+import org.elasticsearch.cluster.node.DiscoveryNode;
 import org.elasticsearch.common.component.AbstractComponent;
 import org.elasticsearch.common.regex.Regex;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.xcontent.XContentHelper;
+import org.elasticsearch.ingest.core.IngestInfo;
 import org.elasticsearch.ingest.core.Pipeline;
 import org.elasticsearch.ingest.core.Processor;
+import org.elasticsearch.ingest.core.ProcessorInfo;
 import org.elasticsearch.ingest.core.TemplateService;
 import org.elasticsearch.script.ScriptService;
 
@@ -47,6 +51,7 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Objects;
 
 public class PipelineStore extends AbstractComponent implements Closeable, ClusterStateListener {
 
@@ -130,8 +135,8 @@ public class PipelineStore extends AbstractComponent implements Closeable, Clust
             pipelines.remove(request.getId());
             ClusterState.Builder newState = ClusterState.builder(currentState);
             newState.metaData(MetaData.builder(currentState.getMetaData())
-                .putCustom(IngestMetadata.TYPE, new IngestMetadata(pipelines))
-                .build());
+                    .putCustom(IngestMetadata.TYPE, new IngestMetadata(pipelines))
+                    .build());
             return newState.build();
         }
     }
@@ -139,15 +144,9 @@ public class PipelineStore extends AbstractComponent implements Closeable, Clust
     /**
      * Stores the specified pipeline definition in the request.
      */
-    public void put(ClusterService clusterService, PutPipelineRequest request, ActionListener<WritePipelineResponse> listener) {
+    public void put(ClusterService clusterService, Map<DiscoveryNode, IngestInfo> ingestInfos, PutPipelineRequest request, ActionListener<WritePipelineResponse> listener) throws Exception {
         // validates the pipeline and processor configuration before submitting a cluster update task:
-        Map<String, Object> pipelineConfig = XContentHelper.convertToMap(request.getSource(), false).v2();
-        try {
-            factory.create(request.getId(), pipelineConfig, processorRegistry);
-        } catch(Exception e) {
-            listener.onFailure(e);
-            return;
-        }
+        validatePipeline(ingestInfos, request);
         clusterService.submitStateUpdateTask("put-pipeline-" + request.getId(), new AckedClusterStateUpdateTask<WritePipelineResponse>(request, listener) {
 
             @Override
@@ -162,6 +161,25 @@ public class PipelineStore extends AbstractComponent implements Closeable, Clust
         });
     }
 
+    void validatePipeline(Map<DiscoveryNode, IngestInfo> ingestInfos, PutPipelineRequest request) throws Exception {
+        if (ingestInfos.isEmpty()) {
+            throw new IllegalStateException("Ingest info is empty");
+        }
+
+        Map<String, Object> pipelineConfig = XContentHelper.convertToMap(request.getSource(), false).v2();
+        Pipeline pipeline = factory.create(request.getId(), pipelineConfig, processorRegistry);
+        List<IllegalArgumentException> exceptions = new ArrayList<>();
+        for (Processor processor : pipeline.flattenAllProcessors()) {
+            for (Map.Entry<DiscoveryNode, IngestInfo> entry : ingestInfos.entrySet()) {
+                if (entry.getValue().containsProcessor(processor.getType()) == false) {
+                    String message = "Processor type [" + processor.getType() + "] is not installed on node [" + entry.getKey() + "]";
+                    exceptions.add(new IllegalArgumentException(message));
+                }
+            }
+        }
+        ExceptionsHelper.rethrowAndSuppress(exceptions);
+    }
+
     ClusterState innerPut(PutPipelineRequest request, ClusterState currentState) {
         IngestMetadata currentIngestMetadata = currentState.metaData().custom(IngestMetadata.TYPE);
         Map<String, PipelineConfiguration> pipelines;

+ 1 - 0
core/src/main/java/org/elasticsearch/ingest/ProcessorsRegistry.java

@@ -21,6 +21,7 @@ package org.elasticsearch.ingest;
 
 import org.apache.lucene.util.IOUtils;
 import org.elasticsearch.ingest.core.Processor;
+import org.elasticsearch.ingest.core.ProcessorInfo;
 import org.elasticsearch.ingest.core.TemplateService;
 
 import java.io.Closeable;

+ 21 - 0
core/src/main/java/org/elasticsearch/ingest/core/CompoundProcessor.java

@@ -20,6 +20,9 @@
 
 package org.elasticsearch.ingest.core;
 
+import org.elasticsearch.common.util.iterable.Iterables;
+
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
@@ -56,6 +59,24 @@ public class CompoundProcessor implements Processor {
         return processors;
     }
 
+    public List<Processor> flattenProcessors() {
+        List<Processor> allProcessors = new ArrayList<>(flattenProcessors(processors));
+        allProcessors.addAll(flattenProcessors(onFailureProcessors));
+        return allProcessors;
+    }
+
+    private static List<Processor> flattenProcessors(List<Processor> processors) {
+        List<Processor> flattened = new ArrayList<>();
+        for (Processor processor : processors) {
+            if (processor instanceof CompoundProcessor) {
+                flattened.addAll(((CompoundProcessor) processor).flattenProcessors());
+            } else {
+                flattened.add(processor);
+            }
+        }
+        return flattened;
+    }
+
     @Override
     public String getType() {
         return "compound";

+ 100 - 0
core/src/main/java/org/elasticsearch/ingest/core/IngestInfo.java

@@ -0,0 +1,100 @@
+/*
+ * Licensed to Elasticsearch under one or more contributor
+ * license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright
+ * ownership. Elasticsearch licenses this file to you under
+ * the Apache License, Version 2.0 (the "License"); you may
+ * not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.elasticsearch.ingest.core;
+
+import org.elasticsearch.common.io.stream.StreamInput;
+import org.elasticsearch.common.io.stream.StreamOutput;
+import org.elasticsearch.common.io.stream.Streamable;
+import org.elasticsearch.common.xcontent.ToXContent;
+import org.elasticsearch.common.xcontent.XContentBuilder;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+
+public class IngestInfo implements Streamable, ToXContent {
+
+    private Set<ProcessorInfo> processors;
+
+    public IngestInfo() {
+        processors = Collections.emptySet();
+    }
+
+    public IngestInfo(List<ProcessorInfo> processors) {
+        this.processors = new LinkedHashSet<>(processors);
+    }
+
+    public Iterable<ProcessorInfo> getProcessors() {
+        return processors;
+    }
+
+    public boolean containsProcessor(String type) {
+        return processors.contains(new ProcessorInfo(type));
+    }
+
+    @Override
+    public void readFrom(StreamInput in) throws IOException {
+        int size = in.readVInt();
+        Set<ProcessorInfo> processors = new LinkedHashSet<>(size);
+        for (int i = 0; i < size; i++) {
+            ProcessorInfo info = new ProcessorInfo();
+            info.readFrom(in);
+            processors.add(info);
+        }
+        this.processors = processors;
+    }
+
+    @Override
+    public void writeTo(StreamOutput out) throws IOException {
+        out.write(processors.size());
+        for (ProcessorInfo info : processors) {
+            info.writeTo(out);
+        }
+    }
+
+    @Override
+    public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
+        builder.startObject("ingest");
+        builder.startArray("processors");
+        for (ProcessorInfo info : processors) {
+            info.toXContent(builder, params);
+        }
+        builder.endArray();
+        builder.endObject();
+        return builder;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+        IngestInfo that = (IngestInfo) o;
+        return Objects.equals(processors, that.processors);
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(processors);
+    }
+}

+ 8 - 0
core/src/main/java/org/elasticsearch/ingest/core/Pipeline.java

@@ -83,6 +83,14 @@ public final class Pipeline {
         return compoundProcessor.getOnFailureProcessors();
     }
 
+    /**
+     * Flattens the normal and on failure processors into a single list. The original order is lost.
+     * This can be useful for pipeline validation purposes.
+     */
+    public List<Processor> flattenAllProcessors() {
+        return compoundProcessor.flattenProcessors();
+    }
+
     public final static class Factory {
 
         public Pipeline create(String id, Map<String, Object> config, ProcessorsRegistry processorRegistry) throws Exception {

+ 81 - 0
core/src/main/java/org/elasticsearch/ingest/core/ProcessorInfo.java

@@ -0,0 +1,81 @@
+/*
+ * Licensed to Elasticsearch under one or more contributor
+ * license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright
+ * ownership. Elasticsearch licenses this file to you under
+ * the Apache License, Version 2.0 (the "License"); you may
+ * not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.elasticsearch.ingest.core;
+
+import org.elasticsearch.common.io.stream.StreamInput;
+import org.elasticsearch.common.io.stream.StreamOutput;
+import org.elasticsearch.common.io.stream.Streamable;
+import org.elasticsearch.common.xcontent.ToXContent;
+import org.elasticsearch.common.xcontent.XContentBuilder;
+
+import java.io.IOException;
+
+public class ProcessorInfo implements Streamable, ToXContent {
+
+    private String type;
+
+    ProcessorInfo() {
+    }
+
+    public ProcessorInfo(String type) {
+        this.type = type;
+    }
+
+    /**
+     * @return The unique processor type
+     */
+    public String getType() {
+        return type;
+    }
+
+    @Override
+    public void readFrom(StreamInput in) throws IOException {
+        this.type = in.readString();
+    }
+
+    @Override
+    public void writeTo(StreamOutput out) throws IOException {
+        out.writeString(this.type);
+    }
+
+    @Override
+    public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
+        builder.startObject();
+        builder.field("type", type);
+        builder.endObject();
+        return null;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+
+        ProcessorInfo that = (ProcessorInfo) o;
+
+        return type.equals(that.type);
+
+    }
+
+    @Override
+    public int hashCode() {
+        return type.hashCode();
+    }
+}

+ 5 - 3
core/src/main/java/org/elasticsearch/node/service/NodeService.java

@@ -132,12 +132,13 @@ public class NodeService extends AbstractComponent implements Closeable {
                 threadPool.info(),
                 transportService.info(),
                 httpServer == null ? null : httpServer.info(),
-                pluginService == null ? null : pluginService.info()
+                pluginService == null ? null : pluginService.info(),
+                ingestService == null ? null : ingestService.info()
         );
     }
 
     public NodeInfo info(boolean settings, boolean os, boolean process, boolean jvm, boolean threadPool,
-                         boolean transport, boolean http, boolean plugin) {
+                         boolean transport, boolean http, boolean plugin, boolean ingest) {
         return new NodeInfo(version, Build.CURRENT, discovery.localNode(), serviceAttributes,
                 settings ? settingsFilter.filter(this.settings) : null,
                 os ? monitorService.osService().info() : null,
@@ -146,7 +147,8 @@ public class NodeService extends AbstractComponent implements Closeable {
                 threadPool ? this.threadPool.info() : null,
                 transport ? transportService.info() : null,
                 http ? (httpServer == null ? null : httpServer.info()) : null,
-                plugin ? (pluginService == null ? null : pluginService.info()) : null
+                plugin ? (pluginService == null ? null : pluginService.info()) : null,
+                ingest ? (ingestService == null ? null : ingestService.info()) : null
         );
     }
 

+ 2 - 1
core/src/main/java/org/elasticsearch/rest/action/admin/cluster/node/info/RestNodesInfoAction.java

@@ -48,7 +48,7 @@ import static org.elasticsearch.rest.RestRequest.Method.GET;
 public class RestNodesInfoAction extends BaseRestHandler {
 
     private final SettingsFilter settingsFilter;
-    private final static Set<String> ALLOWED_METRICS = Sets.newHashSet("http", "jvm", "os", "plugins", "process", "settings", "thread_pool", "transport");
+    private final static Set<String> ALLOWED_METRICS = Sets.newHashSet("http", "jvm", "os", "plugins", "process", "settings", "thread_pool", "transport", "ingest");
 
     @Inject
     public RestNodesInfoAction(Settings settings, RestController controller, Client client, SettingsFilter settingsFilter) {
@@ -101,6 +101,7 @@ public class RestNodesInfoAction extends BaseRestHandler {
             nodesInfoRequest.transport(metrics.contains("transport"));
             nodesInfoRequest.http(metrics.contains("http"));
             nodesInfoRequest.plugins(metrics.contains("plugins"));
+            nodesInfoRequest.ingest(metrics.contains("ingest"));
         }
 
         settingsFilter.addFilterSettingParams(request);

+ 112 - 0
core/src/test/java/org/elasticsearch/ingest/IngestProcessorNotInstalledOnAllNodesIT.java

@@ -0,0 +1,112 @@
+/*
+ * Licensed to Elasticsearch under one or more contributor
+ * license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright
+ * ownership. Elasticsearch licenses this file to you under
+ * the Apache License, Version 2.0 (the "License"); you may
+ * not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.elasticsearch.ingest;
+
+import org.elasticsearch.ElasticsearchParseException;
+import org.elasticsearch.action.ingest.WritePipelineResponse;
+import org.elasticsearch.common.bytes.BytesReference;
+import org.elasticsearch.ingest.core.Pipeline;
+import org.elasticsearch.node.service.NodeService;
+import org.elasticsearch.plugins.Plugin;
+import org.elasticsearch.test.ESIntegTestCase;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+
+import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
+import static org.hamcrest.Matchers.containsString;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.is;
+import static org.hamcrest.Matchers.notNullValue;
+import static org.hamcrest.Matchers.nullValue;
+
+@ESIntegTestCase.ClusterScope(numDataNodes = 0, numClientNodes = 0, scope = ESIntegTestCase.Scope.TEST)
+public class IngestProcessorNotInstalledOnAllNodesIT extends ESIntegTestCase {
+
+    private final BytesReference pipelineSource;
+    private volatile boolean installPlugin;
+
+    public IngestProcessorNotInstalledOnAllNodesIT() throws IOException {
+        pipelineSource = jsonBuilder().startObject()
+                .startArray("processors")
+                    .startObject()
+                        .startObject("test")
+                        .endObject()
+                    .endObject()
+                .endArray()
+                .endObject().bytes();
+    }
+
+    @Override
+    protected Collection<Class<? extends Plugin>> nodePlugins() {
+        return installPlugin ? pluginList(IngestClientIT.IngestPlugin.class) : Collections.emptyList();
+    }
+
+    @Override
+    protected Collection<Class<? extends Plugin>> getMockPlugins() {
+        return Collections.singletonList(TestSeedPlugin.class);
+    }
+
+    public void testFailPipelineCreation() throws Exception {
+        installPlugin = true;
+        internalCluster().startNode();
+        installPlugin = false;
+        internalCluster().startNode();
+
+        try {
+            client().admin().cluster().preparePutPipeline("_id", pipelineSource).get();
+            fail("exception expected");
+        } catch (IllegalArgumentException e) {
+            assertThat(e.getMessage(), containsString("Processor type [test] is not installed on node"));
+        }
+    }
+
+    public void testFailPipelineCreationProcessorNotInstalledOnMasterNode() throws Exception {
+        internalCluster().startNode();
+        installPlugin = true;
+        internalCluster().startNode();
+
+        try {
+            client().admin().cluster().preparePutPipeline("_id", pipelineSource).get();
+            fail("exception expected");
+        } catch (ElasticsearchParseException e) {
+            assertThat(e.getMessage(), equalTo("No processor type exists with name [test]"));
+        }
+    }
+
+    // If there is pipeline defined and a node joins that doesn't have the processor installed then
+    // that pipeline can't be used on this node.
+    public void testFailStartNode() throws Exception {
+        installPlugin = true;
+        String node1 = internalCluster().startNode();
+
+        WritePipelineResponse response = client().admin().cluster().preparePutPipeline("_id", pipelineSource).get();
+        assertThat(response.isAcknowledged(), is(true));
+        Pipeline pipeline = internalCluster().getInstance(NodeService.class, node1).getIngestService().getPipelineStore().get("_id");
+        assertThat(pipeline, notNullValue());
+
+        installPlugin = false;
+        String node2 = internalCluster().startNode();
+        pipeline = internalCluster().getInstance(NodeService.class, node2).getIngestService().getPipelineStore().get("_id");
+        assertThat(pipeline, nullValue());
+    }
+
+}

+ 43 - 0
core/src/test/java/org/elasticsearch/ingest/PipelineStoreTests.java

@@ -21,24 +21,32 @@ package org.elasticsearch.ingest;
 
 import org.elasticsearch.ElasticsearchParseException;
 import org.elasticsearch.ResourceNotFoundException;
+import org.elasticsearch.Version;
 import org.elasticsearch.action.ingest.DeletePipelineRequest;
 import org.elasticsearch.action.ingest.PutPipelineRequest;
 import org.elasticsearch.cluster.ClusterName;
 import org.elasticsearch.cluster.ClusterState;
 import org.elasticsearch.cluster.metadata.MetaData;
+import org.elasticsearch.cluster.node.DiscoveryNode;
 import org.elasticsearch.common.bytes.BytesArray;
 import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.common.transport.LocalTransportAddress;
+import org.elasticsearch.ingest.core.IngestInfo;
 import org.elasticsearch.ingest.core.Pipeline;
+import org.elasticsearch.ingest.core.ProcessorInfo;
+import org.elasticsearch.ingest.processor.RemoveProcessor;
 import org.elasticsearch.ingest.processor.SetProcessor;
 import org.elasticsearch.test.ESTestCase;
 import org.junit.Before;
 
+import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
 import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.instanceOf;
 import static org.hamcrest.Matchers.is;
 import static org.hamcrest.Matchers.notNullValue;
 import static org.hamcrest.Matchers.nullValue;
@@ -52,6 +60,7 @@ public class PipelineStoreTests extends ESTestCase {
         store = new PipelineStore(Settings.EMPTY);
         ProcessorsRegistry.Builder registryBuilder = new ProcessorsRegistry.Builder();
         registryBuilder.registerProcessor("set", (templateService, registry) -> new SetProcessor.Factory(TestTemplateService.instance()));
+        registryBuilder.registerProcessor("remove", (templateService, registry) -> new RemoveProcessor.Factory(TestTemplateService.instance()));
         store.buildProcessorFactoryRegistry(registryBuilder, null);
     }
 
@@ -197,4 +206,38 @@ public class PipelineStoreTests extends ESTestCase {
         assertThat(pipeline, nullValue());
     }
 
+    public void testValidate() throws Exception {
+        PutPipelineRequest putRequest = new PutPipelineRequest("_id", new BytesArray("{\"processors\": [{\"set\" : {\"field\": \"_field\", \"value\": \"_value\"}},{\"remove\" : {\"field\": \"_field\"}}]}"));
+
+        DiscoveryNode node1 = new DiscoveryNode("_node_id1", new LocalTransportAddress("_id"), Version.CURRENT);
+        DiscoveryNode node2 = new DiscoveryNode("_node_id2", new LocalTransportAddress("_id"), Version.CURRENT);
+        Map<DiscoveryNode, IngestInfo> ingestInfos = new HashMap<>();
+        ingestInfos.put(node1, new IngestInfo(Arrays.asList(new ProcessorInfo("set"), new ProcessorInfo("remove"))));
+        ingestInfos.put(node2, new IngestInfo(Arrays.asList(new ProcessorInfo("set"))));
+
+        try {
+            store.validatePipeline(ingestInfos, putRequest);
+            fail("exception expected");
+        } catch (IllegalArgumentException e) {
+            assertThat(e.getMessage(), equalTo("Processor type [remove] is not installed on node [{_node_id2}{local}{local[_id]}]"));
+        }
+
+        ingestInfos.put(node2, new IngestInfo(Arrays.asList(new ProcessorInfo("set"), new ProcessorInfo("remove"))));
+        store.validatePipeline(ingestInfos, putRequest);
+    }
+
+    public void testValidateNoIngestInfo() throws Exception {
+        PutPipelineRequest putRequest = new PutPipelineRequest("_id", new BytesArray("{\"processors\": [{\"set\" : {\"field\": \"_field\", \"value\": \"_value\"}}]}"));
+        try {
+            store.validatePipeline(Collections.emptyMap(), putRequest);
+            fail("exception expected");
+        } catch (IllegalStateException e) {
+            assertThat(e.getMessage(), equalTo("Ingest info is empty"));
+        }
+
+        DiscoveryNode discoveryNode = new DiscoveryNode("_node_id", new LocalTransportAddress("_id"), Version.CURRENT);
+        IngestInfo ingestInfo = new IngestInfo(Collections.singletonList(new ProcessorInfo("set")));
+        store.validatePipeline(Collections.singletonMap(discoveryNode, ingestInfo), putRequest);
+    }
+
 }

+ 12 - 0
core/src/test/java/org/elasticsearch/ingest/core/PipelineFactoryTests.java

@@ -23,11 +23,14 @@ import org.elasticsearch.ElasticsearchParseException;
 import org.elasticsearch.ingest.ProcessorsRegistry;
 import org.elasticsearch.ingest.TestProcessor;
 import org.elasticsearch.ingest.TestTemplateService;
+import org.elasticsearch.ingest.processor.FailProcessor;
+import org.elasticsearch.ingest.processor.SetProcessor;
 import org.elasticsearch.test.ESTestCase;
 
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 import java.util.prefs.PreferencesFactory;
 
@@ -115,6 +118,15 @@ public class PipelineFactoryTests extends ESTestCase {
         assertThat(pipeline.getProcessors().get(0).getType(), equalTo("compound"));
     }
 
+    public void testFlattenProcessors() throws Exception {
+        TestProcessor testProcessor = new TestProcessor(ingestDocument -> {});
+        CompoundProcessor processor1 = new CompoundProcessor(testProcessor, testProcessor);
+        CompoundProcessor processor2 = new CompoundProcessor(Collections.singletonList(testProcessor), Collections.singletonList(testProcessor));
+        Pipeline pipeline = new Pipeline("_id", "_description", new CompoundProcessor(processor1, processor2));
+        List<Processor> flattened = pipeline.flattenAllProcessors();
+        assertThat(flattened.size(), equalTo(4));
+    }
+
     private ProcessorsRegistry createProcessorRegistry(Map<String, Processor.Factory> processorRegistry) {
         ProcessorsRegistry.Builder builder = new ProcessorsRegistry.Builder();
         for (Map.Entry<String, Processor.Factory> entry : processorRegistry.entrySet()) {

+ 4 - 1
core/src/test/java/org/elasticsearch/nodesinfo/NodeInfoStreamingTests.java

@@ -33,6 +33,7 @@ import org.elasticsearch.common.transport.TransportAddress;
 import org.elasticsearch.common.xcontent.ToXContent;
 import org.elasticsearch.common.xcontent.XContentBuilder;
 import org.elasticsearch.http.HttpInfo;
+import org.elasticsearch.ingest.core.IngestInfo;
 import org.elasticsearch.monitor.jvm.JvmInfo;
 import org.elasticsearch.monitor.os.DummyOsInfo;
 import org.elasticsearch.monitor.os.OsInfo;
@@ -90,6 +91,7 @@ public class NodeInfoStreamingTests extends ESTestCase {
         compareJsonOutput(nodeInfo.getNode(), readNodeInfo.getNode());
         compareJsonOutput(nodeInfo.getOs(), readNodeInfo.getOs());
         comparePluginsAndModules(nodeInfo, readNodeInfo);
+        compareJsonOutput(nodeInfo.getIngest(), readNodeInfo.getIngest());
     }
 
     private void comparePluginsAndModules(NodeInfo nodeInfo, NodeInfo readNodeInfo) throws IOException {
@@ -135,6 +137,7 @@ public class NodeInfoStreamingTests extends ESTestCase {
         PluginsAndModules plugins = new PluginsAndModules();
         plugins.addModule(DummyPluginInfo.INSTANCE);
         plugins.addPlugin(DummyPluginInfo.INSTANCE);
-        return new NodeInfo(VersionUtils.randomVersion(random()), build, node, serviceAttributes, settings, osInfo, process, jvm, threadPoolInfo, transport, htttpInfo, plugins);
+        IngestInfo ingestInfo = new IngestInfo();
+        return new NodeInfo(VersionUtils.randomVersion(random()), build, node, serviceAttributes, settings, osInfo, process, jvm, threadPoolInfo, transport, htttpInfo, plugins, ingestInfo);
     }
 }

+ 69 - 1
docs/reference/cluster/nodes-info.asciidoc

@@ -17,7 +17,7 @@ The second command selectively retrieves nodes information of only
 
 By default, it just returns all attributes and core settings for a node.
 It also allows to get only information on `settings`, `os`, `process`, `jvm`,
-`thread_pool`, `transport`, `http` and `plugins`:
+`thread_pool`, `transport`, `http`, `plugins` and `ingest`:
 
 [source,js]
 --------------------------------------------------
@@ -122,3 +122,71 @@ The result will look similar to:
   }
 }
 --------------------------------------------------
+
+[float]
+[[ingest-info]]
+==== Ingest information
+
+`ingest` - if set, the result will contain details about the available
+processors per node:
+
+* `type`: the processor type
+
+The result will look similar to:
+
+[source,js]
+--------------------------------------------------
+{
+  "cluster_name": "elasticsearch",
+  "nodes": {
+    "O70_wBv6S9aPPcAKdSUBtw": {
+      "ingest": {
+        "processors": [
+          {
+            "type": "date"
+          },
+          {
+            "type": "uppercase"
+          },
+          {
+            "type": "set"
+          },
+          {
+            "type": "lowercase"
+          },
+          {
+            "type": "gsub"
+          },
+          {
+            "type": "convert"
+          },
+          {
+            "type": "remove"
+          },
+          {
+            "type": "fail"
+          },
+          {
+            "type": "foreach"
+          },
+          {
+            "type": "split"
+          },
+          {
+            "type": "trim"
+          },
+          {
+            "type": "rename"
+          },
+          {
+            "type": "join"
+          },
+          {
+            "type": "append"
+          }
+        ]
+      }
+    }
+  }
+}
+--------------------------------------------------

+ 14 - 0
docs/reference/ingest/ingest-node.asciidoc

@@ -620,6 +620,20 @@ but is very useful for bookkeeping and tracing errors to specific processors.
 
 See <<handling-failure-in-pipelines>> to learn more about the `on_failure` field and error handling in pipelines.
 
+The <<ingest-info,node info API>> can be used to figure out what processors are available in a cluster.
+The <<ingest-info,node info API>> will provide a per node list of what processors are available.
+
+Custom processors must be installed on all nodes. The put pipeline API will fail if a processor specified in a pipeline
+doesn't exist on all nodes. If you rely on custom processor plugins make sure to mark these plugins as mandatory by adding
+`plugin.mandatory` setting to the `config/elasticsearch.yml` file, for example:
+
+[source,yaml]
+--------------------------------------------------
+plugin.mandatory: ingest-attachment,ingest-geoip
+--------------------------------------------------
+
+A node will not start if either of these plugins are not available.
+
 [[append-procesesor]]
 === Append Processor
 Appends one or more values to an existing array if the field already exists and it is an array.

+ 8 - 2
plugins/ingest-attachment/src/test/resources/rest-api-spec/test/ingest_attachment/10_basic.yaml

@@ -1,5 +1,11 @@
 "Ingest attachment plugin installed":
     - do:
-        cluster.stats: {}
+        cluster.state: {}
 
-    - match:  { nodes.plugins.0.name: ingest-attachment }
+    - set: {master_node: master}
+
+    - do:
+        nodes.info: {}
+
+    - match:  { nodes.$master.plugins.0.name: ingest-attachment  }
+    - match:  { nodes.$master.ingest.processors.11.type: attachment }

+ 8 - 2
plugins/ingest-geoip/src/test/resources/rest-api-spec/test/ingest_geoip/10_basic.yaml

@@ -1,5 +1,11 @@
 "Ingest plugin installed":
     - do:
-        cluster.stats: {}
+        cluster.state: {}
 
-    - match:  { nodes.plugins.0.name: ingest-geoip  }
+    - set: {master_node: master}
+
+    - do:
+        nodes.info: {}
+
+    - match:  { nodes.$master.plugins.0.name: ingest-geoip  }
+    - match:  { nodes.$master.ingest.processors.3.type: geoip }

+ 1 - 1
rest-api-spec/src/main/resources/rest-api-spec/api/nodes.info.json

@@ -12,7 +12,7 @@
         },
         "metric": {
           "type": "list",
-          "options": ["settings", "os", "process", "jvm", "thread_pool", "transport", "http", "plugins"],
+          "options": ["settings", "os", "process", "jvm", "thread_pool", "transport", "http", "plugins", "ingest"],
           "description": "A comma-separated list of metrics you wish returned. Leave empty to return all."
         }
       },

+ 25 - 0
rest-api-spec/src/main/resources/rest-api-spec/test/ingest/10_crud.yaml

@@ -1,3 +1,28 @@
+---
+"Check availability of default processors":
+  - do:
+      cluster.state: {}
+
+  - set: {master_node: master}
+
+  - do:
+      nodes.info: {}
+
+  - match:  { nodes.$master.ingest.processors.0.type: date }
+  - match:  { nodes.$master.ingest.processors.1.type: uppercase }
+  - match:  { nodes.$master.ingest.processors.2.type: set }
+  - match:  { nodes.$master.ingest.processors.3.type: lowercase }
+  - match:  { nodes.$master.ingest.processors.4.type: gsub }
+  - match:  { nodes.$master.ingest.processors.5.type: convert }
+  - match:  { nodes.$master.ingest.processors.6.type: remove }
+  - match:  { nodes.$master.ingest.processors.7.type: fail }
+  - match:  { nodes.$master.ingest.processors.8.type: foreach }
+  - match:  { nodes.$master.ingest.processors.9.type: split }
+  - match:  { nodes.$master.ingest.processors.10.type: trim }
+  - match:  { nodes.$master.ingest.processors.11.type: rename }
+  - match:  { nodes.$master.ingest.processors.12.type: join }
+  - match:  { nodes.$master.ingest.processors.13.type: append }
+
 ---
 "Test basic pipeline crud":
   - do: