Browse Source

Redact Ingest Processor (#92951)

The Redact processor uses the Grok rules engine to
redact text in the input document that matches the
Grok pattern. For example Email or IP addresses can
be redacted using the definitions from the standard
Grok pattern bank. New patterns can be defined in
the processor configuration
David Kyle 2 years ago
parent
commit
b588d2ddd7

+ 5 - 0
docs/changelog/92951.yaml

@@ -0,0 +1,5 @@
+pr: 92951
+summary: Redact Ingest Processor
+area: Ingest Node
+type: feature
+issues: []

+ 1 - 0
docs/reference/ingest/processors.asciidoc

@@ -60,6 +60,7 @@ include::processors/kv.asciidoc[]
 include::processors/lowercase.asciidoc[]
 include::processors/network-direction.asciidoc[]
 include::processors/pipeline.asciidoc[]
+include::processors/redact.asciidoc[]
 include::processors/registered-domain.asciidoc[]
 include::processors/remove.asciidoc[]
 include::processors/rename.asciidoc[]

+ 230 - 0
docs/reference/ingest/processors/redact.asciidoc

@@ -0,0 +1,230 @@
+[[redact-processor]]
+=== Redact processor
+++++
+<titleabbrev>Redact</titleabbrev>
+++++
+
+The Redact processor obscures portions of text in the input document
+matching the given Grok patterns. A Grok pattern is like a regular
+expression with a named capturing group: text that matches the regular
+expression is replaced with the capture name. {es} comes packaged with
+a number of useful predefined {es-repo}blob/{branch}/libs/grok/src/main/resources/patterns/ecs-v1[patterns].
+If one of those does not suit your needs, create a new pattern with a
+custom pattern definition. The Redact processor replaces every occurrence
+of a match. If there are multiple matches all will be replaced with the
+capture name.
+
+The Redact processor is compatible with {ecs-ref}/ecs-field-reference.html[Elastic Common Schema (ECS)]
+patterns. Legacy Grok patterns are not supported.
+
+[[using-redact]]
+
+==== Using the Redact processor in a pipeline
+
+[[redact-options]]
+.Redact Options
+[options="header"]
+|======
+| Name                   | Required  | Default             | Description
+| `field`                | yes       | -                   | The field to be redacted
+| `patterns`             | yes       | -                   | A list of grok expressions to match and redact named captures with
+| `pattern_definitions`  | no        | -                   | A map of pattern-name and pattern tuples defining custom patterns to be used by the processor. Patterns matching existing names will override the pre-existing definition
+| `prefix`               | no        | <                   | Start a redacted section with this token
+| `suffix`               | no        | >                   | End a redacted section with this token
+| `ignore_missing`       | no        | false               | If `true` and `field` does not exist or is `null`, the processor quietly exits without modifying the document
+include::common-options.asciidoc[]
+|======
+
+In this example the predefined `IP` Grok pattern is used to match
+and redact an IP addresses from the `message` text field. The pipeline
+is tested using the Simulate API.
+
+[source,console]
+--------------------------------------------------
+POST _ingest/pipeline/_simulate
+{
+  "pipeline": {
+    "description" : "Hide my IP",
+    "processors": [
+      {
+        "redact": {
+          "field": "message",
+          "patterns": ["%{IP:client}"]
+        }
+      }
+    ]
+  },
+  "docs":[
+    {
+      "_source": {
+        "message": "55.3.244.1 GET /index.html 15824 0.043"
+      }
+    }
+  ]
+}
+--------------------------------------------------
+
+The document in the response still contains the `message` field
+but now the IP address `55.3.244.1` is replaced by the text `<client>`.
+
+[source,console-result]
+--------------------------------------------------
+{
+  "docs": [
+    {
+      "doc": {
+        "_index": "_index",
+        "_id": "_id",
+        "_version": "-3",
+        "_source": {
+          "message": "<client> GET /index.html 15824 0.043"
+        },
+        "_ingest": {
+          "timestamp": "2023-02-01T16:08:39.419056008Z"
+        }
+      }
+    }
+  ]
+}
+--------------------------------------------------
+// TESTRESPONSE[s/2023-02-01T16:08:39.419056008Z/$body.docs.0.doc._ingest.timestamp/]
+
+The IP address is replaced with the word `client` because that is what is
+specified in the Grok pattern `%{IP:client}`. The `<` and `>` tokens which
+surround the pattern name are configurable using the `prefix` and `suffix` options.
+
+The next example defines multiple patterns both of which are replaced
+with the word `REDACTED` and the prefix and suffix tokens are set to `*`
+
+[source,console]
+--------------------------------------------------
+POST _ingest/pipeline/_simulate
+{
+  "pipeline": {
+    "description": "Hide my IP",
+    "processors": [
+      {
+        "redact": {
+          "field": "message",
+          "patterns": [
+            "%{IP:REDACTED}",
+            "%{EMAILADDRESS:REDACTED}"
+          ],
+          "prefix": "*",
+          "suffix": "*"
+        }
+      }
+    ]
+  },
+  "docs": [
+    {
+      "_source": {
+        "message": "55.3.244.1 GET /index.html 15824 0.043 test@elastic.co"
+      }
+    }
+  ]
+}
+--------------------------------------------------
+
+In the response both the IP `55.3.244.1` and email address `test@elastic.co`
+have been replaced by `*REDACTED*`.
+
+[source,console-result]
+--------------------------------------------------
+{
+  "docs": [
+    {
+      "doc": {
+        "_index": "_index",
+        "_id": "_id",
+        "_version": "-3",
+        "_source": {
+          "message": "*REDACTED* GET /index.html 15824 0.043 *REDACTED*"
+        },
+        "_ingest": {
+          "timestamp": "2023-02-01T16:53:14.560005377Z"
+        }
+      }
+    }
+  ]
+}
+--------------------------------------------------
+// TESTRESPONSE[s/2023-02-01T16:53:14.560005377Z/$body.docs.0.doc._ingest.timestamp/]
+
+[[redact-custom-patterns]]
+==== Custom patterns
+
+If one of the existing Grok {es-repo}blob/{branch}/libs/grok/src/main/resources/patterns/ecs-v1[patterns]
+does not fit your requirements custom patterns can be added with the
+`pattern_definitions` option. New patterns definitions are composed of
+a pattern name and the pattern itself. The pattern may be a regular
+expression or reference existing Grok patterns.
+
+This example defines the custom pattern `GITHUB_NAME` to match
+GitHub usernames. The pattern definition uses the existing
+`USERNAME` Grok https://github.com/elastic/elasticsearch/blob/{branch}/libs/grok/src/main/resources/patterns/ecs-v1/grok-patterns[pattern] prefixed by the
+literal `@`.
+
+NOTE: The {kibana-ref}/xpack-grokdebugger.html[Grok Debugger] is a really
+useful tool for building custom patterns.
+
+[source,console]
+--------------------------------------------------
+POST _ingest/pipeline/_simulate
+{
+  "pipeline": {
+    "processors": [
+      {
+        "redact": {
+          "field": "message",
+          "patterns": [
+            "%{GITHUB_NAME:GITHUB_NAME}"
+          ],
+          "pattern_definitions": {
+            "GITHUB_NAME": "@%{USERNAME}"
+          }
+        }
+      }
+    ]
+  },
+  "docs": [
+    {
+      "_source": {
+        "message": "@elastic-data-management the PR is ready for review"
+      }
+    }
+  ]
+}
+--------------------------------------------------
+
+The username is redacted in the response.
+
+[source,console-result]
+--------------------------------------------------
+{
+  "docs": [
+    {
+      "doc": {
+        "_index": "_index",
+        "_id": "_id",
+        "_version": "-3",
+        "_source": {
+          "message": "<GITHUB_NAME> the PR is ready for review"
+        },
+        "_ingest": {
+          "timestamp": "2023-02-01T16:53:14.560005377Z"
+        }
+      }
+    }
+  ]
+}
+--------------------------------------------------
+// TESTRESPONSE[s/2023-02-01T16:53:14.560005377Z/$body.docs.0.doc._ingest.timestamp/]
+
+[[grok-watchdog-redact]]
+==== Grok watchdog
+
+The watchdog interrupts expressions that take too long to execute.
+When interrupted, the Redact processor fails with an error.
+The same <<grok-watchdog-options, settings>> that control the
+Grok Watchdog timeout also apply to the Redact processor.

+ 4 - 0
libs/grok/src/main/java/org/elasticsearch/grok/Grok.java

@@ -325,6 +325,10 @@ public final class Grok {
         return captureConfig;
     }
 
+    public Regex getCompiledExpression() {
+        return compiledExpression;
+    }
+
     /**
      * Load built-in patterns.
      */

+ 1 - 0
modules/ingest-common/src/main/java/module-info.java

@@ -17,6 +17,7 @@ module org.elasticsearch.ingest.common {
     requires org.apache.httpcomponents.httpclient;
     requires org.apache.logging.log4j;
     requires org.apache.lucene.analysis.common;
+    requires org.jruby.joni;
 
     exports org.elasticsearch.ingest.common; // for painless
 

+ 4 - 2
modules/ingest-common/src/main/java/org/elasticsearch/ingest/common/IngestCommonPlugin.java

@@ -53,6 +53,7 @@ public class IngestCommonPlugin extends Plugin implements ActionPlugin, IngestPl
 
     @Override
     public Map<String, Processor.Factory> getProcessors(Processor.Parameters parameters) {
+        var matcherWatchdog = createGrokThreadWatchdog(parameters);
         return Map.ofEntries(
             entry(DateProcessor.TYPE, new DateProcessor.Factory(parameters.scriptService)),
             entry(SetProcessor.TYPE, new SetProcessor.Factory(parameters.scriptService)),
@@ -70,7 +71,7 @@ public class IngestCommonPlugin extends Plugin implements ActionPlugin, IngestPl
             entry(ForEachProcessor.TYPE, new ForEachProcessor.Factory(parameters.scriptService)),
             entry(DateIndexNameProcessor.TYPE, new DateIndexNameProcessor.Factory(parameters.scriptService)),
             entry(SortProcessor.TYPE, new SortProcessor.Factory()),
-            entry(GrokProcessor.TYPE, new GrokProcessor.Factory(createGrokThreadWatchdog(parameters))),
+            entry(GrokProcessor.TYPE, new GrokProcessor.Factory(matcherWatchdog)),
             entry(ScriptProcessor.TYPE, new ScriptProcessor.Factory(parameters.scriptService)),
             entry(DotExpanderProcessor.TYPE, new DotExpanderProcessor.Factory()),
             entry(JsonProcessor.TYPE, new JsonProcessor.Factory()),
@@ -86,7 +87,8 @@ public class IngestCommonPlugin extends Plugin implements ActionPlugin, IngestPl
             entry(NetworkDirectionProcessor.TYPE, new NetworkDirectionProcessor.Factory(parameters.scriptService)),
             entry(CommunityIdProcessor.TYPE, new CommunityIdProcessor.Factory()),
             entry(FingerprintProcessor.TYPE, new FingerprintProcessor.Factory()),
-            entry(RegisteredDomainProcessor.TYPE, new RegisteredDomainProcessor.Factory())
+            entry(RegisteredDomainProcessor.TYPE, new RegisteredDomainProcessor.Factory()),
+            entry(RedactProcessor.TYPE, new RedactProcessor.Factory(matcherWatchdog))
         );
     }
 

+ 380 - 0
modules/ingest-common/src/main/java/org/elasticsearch/ingest/common/RedactProcessor.java

@@ -0,0 +1,380 @@
+/*
+ * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
+ * or more contributor license agreements. Licensed under the Elastic License
+ * 2.0 and the Server Side Public License, v 1; you may not use this file except
+ * in compliance with, at your election, the Elastic License 2.0 or the Server
+ * Side Public License, v 1.
+ */
+
+package org.elasticsearch.ingest.common;
+
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+import org.elasticsearch.ElasticsearchTimeoutException;
+import org.elasticsearch.grok.Grok;
+import org.elasticsearch.grok.GrokCaptureExtracter;
+import org.elasticsearch.grok.MatcherWatchdog;
+import org.elasticsearch.ingest.AbstractProcessor;
+import org.elasticsearch.ingest.ConfigurationUtils;
+import org.elasticsearch.ingest.IngestDocument;
+import org.elasticsearch.ingest.Processor;
+import org.joni.Matcher;
+import org.joni.Option;
+import org.joni.Region;
+
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.elasticsearch.ingest.ConfigurationUtils.newConfigurationException;
+
+/**
+ * An ingest processor for redacting (obscuring) input text.
+ * Uses Grok patterns to match text in the input, matches are
+ * then replaced with redacted text.
+ */
+public class RedactProcessor extends AbstractProcessor {
+
+    public static final String TYPE = "redact";
+
+    private static final Logger logger = LogManager.getLogger(RedactProcessor.class);
+
+    private static final String DEFAULT_REDACTED_START = "<";
+    private static final String DEFAULT_REDACTED_END = ">";
+
+    private final String redactField;
+    private final List<Grok> groks;
+    private final boolean ignoreMissing;
+    private final String redactedStartToken;
+    private final String redactedEndToken;
+
+    RedactProcessor(
+        String tag,
+        String description,
+        Map<String, String> patternBank,
+        List<String> matchPatterns,
+        String redactField,
+        boolean ignoreMissing,
+        String redactedStartToken,
+        String redactedEndToken,
+        MatcherWatchdog matcherWatchdog
+    ) {
+        super(tag, description);
+        this.redactField = redactField;
+        this.redactedStartToken = redactedStartToken;
+        this.redactedEndToken = redactedEndToken;
+        this.groks = new ArrayList<>(matchPatterns.size());
+        for (var matchPattern : matchPatterns) {
+            this.groks.add(new Grok(patternBank, matchPattern, matcherWatchdog, logger::debug));
+        }
+        this.ignoreMissing = ignoreMissing;
+        // Joni warnings are only emitted on an attempt to match, and the warning emitted for every call to match which is too verbose
+        // so here we emit a warning (if there is one) to the logfile at warn level on construction / processor creation.
+        if (matchPatterns.isEmpty() == false) {
+            new Grok(patternBank, matchPatterns.get(0), matcherWatchdog, logger::warn).match("___nomatch___");
+        }
+    }
+
+    @Override
+    public IngestDocument execute(IngestDocument ingestDocument) {
+        // Call with ignoreMissing = true so getFieldValue does not throw
+        final String fieldValue = ingestDocument.getFieldValue(redactField, String.class, true);
+
+        if (fieldValue == null && ignoreMissing) {
+            return ingestDocument;
+        } else if (fieldValue == null) {
+            throw new IllegalArgumentException("field [" + redactField + "] is null or missing");
+        }
+
+        try {
+            String redacted = matchRedact(fieldValue, groks, redactedStartToken, redactedEndToken);
+            ingestDocument.setFieldValue(redactField, redacted);
+            return ingestDocument;
+        } catch (RuntimeException e) {
+            // grok throws a RuntimeException when the watchdog interrupts the match
+            throw new ElasticsearchTimeoutException("Grok pattern matching timed out", e);
+        }
+    }
+
+    @Override
+    public String getType() {
+        return TYPE;
+    }
+
+    List<Grok> getGroks() {
+        return groks;
+    }
+
+    // exposed for testing
+    static String matchRedact(String fieldValue, List<Grok> groks) {
+        return matchRedact(fieldValue, groks, DEFAULT_REDACTED_START, DEFAULT_REDACTED_END);
+    }
+
+    /**
+     * Finds all matches for each of the {@code groks} in {@code fieldValue} and
+     * replaces the matched text with Grok pattern name.
+     *
+     * @param fieldValue Text to redact
+     * @param groks Groks to match
+     * @param redactedStartToken Matched and redacted regions are started with this token
+     * @param redactedEndToken Matched and redacted regions are ended with this token
+     * @return The redacted text
+     */
+    static String matchRedact(String fieldValue, List<Grok> groks, String redactedStartToken, String redactedEndToken) {
+        byte[] utf8Bytes = fieldValue.getBytes(StandardCharsets.UTF_8);
+
+        RegionTrackingMatchExtractor extractor = new RegionTrackingMatchExtractor();
+        for (var grok : groks) {
+            String patternName = grok.captureConfig().get(0).name();
+            extractor.setCurrentPatternName(patternName);
+            matchRepeat(grok, utf8Bytes, extractor);
+        }
+
+        if (extractor.replacementPositions.isEmpty()) {
+            // no matches, nothing to redact
+            return fieldValue;
+        }
+        return extractor.redactMatches(utf8Bytes, redactedStartToken, redactedEndToken);
+    }
+
+    private static void matchRepeat(Grok grok, byte[] utf8Bytes, RegionTrackingMatchExtractor extractor) {
+        Matcher matcher = grok.getCompiledExpression().matcher(utf8Bytes, 0, utf8Bytes.length);
+        int result;
+        int offset = 0;
+        int length = utf8Bytes.length;
+
+        do {
+            result = matcher.search(offset, length, Option.DEFAULT);
+            if (result < 0) {
+                break;
+            }
+
+            extractor.extract(utf8Bytes, 0, matcher.getEagerRegion());
+
+            if (matcher.getEnd() == offset) {
+                ++offset;
+            } else {
+                offset = matcher.getEnd();
+            }
+
+        } while (offset != length);
+    }
+
+    /**
+     * A Grok capture extractor which tracks matched regions
+     * and the Grok pattern name for redaction later.
+     */
+    static class RegionTrackingMatchExtractor implements GrokCaptureExtracter {
+
+        static class Replacement {
+            int start;
+            int end;
+            final String patternName;
+
+            Replacement(int start, int end, String patternName) {
+                this.start = start;
+                this.end = end;
+                this.patternName = patternName;
+            }
+
+            int length() {
+                return end - start;
+            }
+
+            @Override
+            public String toString() {
+                return "Replacement{" + "start=" + start + ", end=" + end + ", patternName='" + patternName + '\'' + '}';
+            }
+        }
+
+        private final List<Replacement> replacementPositions;
+        private String patternName;
+
+        RegionTrackingMatchExtractor() {
+            replacementPositions = new ArrayList<>();
+        }
+
+        void setCurrentPatternName(String patternName) {
+            this.patternName = patternName;
+        }
+
+        @Override
+        public void extract(byte[] utf8Bytes, int offset, Region region) {
+            assert patternName != null;
+
+            int number = 0;
+            int matchOffset = offset + region.beg[number];
+            int matchEnd = offset + region.end[number];
+            replacementPositions.add(new Replacement(matchOffset, matchEnd, patternName));
+        }
+
+        /**
+         * Replaces the matched regions in the original input text (passed as UTF8 bytes)
+         * with the Grok matches. Each match is replaced by {@code redactStartToken}
+         * followed by the Grok pattern name then suffixed by {@code redactEndToken}.
+         *
+         * Special care is taken to detect and manage regions that overlap, i.e. where
+         * more than 1 Grok pattern has matched a piece of text. Where regions overlap
+         * the pattern name of the longest matching regions is used for the replacement.
+         *
+         * @param utf8Bytes The original text as UTF8
+         * @param redactStartToken The token to prefix the matched and redacted pattern name with
+         * @param redactEndToken The token to suffix the matched and redacted pattern name with
+         * @return A String with the matched regions redacted
+         */
+        String redactMatches(byte[] utf8Bytes, String redactStartToken, String redactEndToken) {
+            var merged = mergeOverlappingReplacements(replacementPositions);
+            int longestPatternName = merged.stream().mapToInt(r -> r.patternName.getBytes(StandardCharsets.UTF_8).length).max().getAsInt();
+
+            int maxPossibleLength = longestPatternName * merged.size() + utf8Bytes.length;
+            byte[] redact = new byte[maxPossibleLength];
+
+            int readOffset = 0;
+            int writeOffset = 0;
+            for (var rep : merged) {
+                int numBytesToWrite = rep.start - readOffset;
+                System.arraycopy(utf8Bytes, readOffset, redact, writeOffset, numBytesToWrite);
+                readOffset = rep.end;
+
+                writeOffset = writeOffset + numBytesToWrite;
+
+                byte[] replacementText = (redactStartToken + rep.patternName + redactEndToken).getBytes(StandardCharsets.UTF_8);
+                System.arraycopy(replacementText, 0, redact, writeOffset, replacementText.length);
+
+                writeOffset = writeOffset + replacementText.length;
+            }
+
+            int numBytesToWrite = utf8Bytes.length - readOffset;
+            System.arraycopy(utf8Bytes, readOffset, redact, writeOffset, numBytesToWrite);
+            writeOffset = writeOffset + numBytesToWrite;
+
+            return new String(redact, 0, writeOffset, StandardCharsets.UTF_8);
+        }
+
+        /**
+         * If {@code replacementPositions} contains overlapping regions
+         * merge the overlaps.
+         *
+         * The strategy is to first sort the replacements by start position
+         * then find and merge overlapping regions.
+         * @param replacementPositions Found replacements, some of which may overlap
+         * @return List of replacements ordered by start position
+         */
+        static List<Replacement> mergeOverlappingReplacements(List<Replacement> replacementPositions) {
+            if (replacementPositions.size() == 1) {
+                return replacementPositions;
+            }
+
+            List<Replacement> result = new ArrayList<>();
+            // sort by start position
+            replacementPositions.sort(Comparator.comparingInt(a -> a.start));
+            int current = 0;
+            int next = 1;
+            while (current < replacementPositions.size()) {
+                var head = replacementPositions.get(current);
+                if (next >= replacementPositions.size() || head.end < replacementPositions.get(next).start) {
+                    // Either current points to the last item in the list or
+                    // there is no overlap and nothing to merge.
+                    // Add the item to the result list
+                    result.add(head);
+                } else {
+                    // Overlapping regions and the overlaps can be transitive
+                    int previousRegionEnd;
+                    do {
+                        previousRegionEnd = replacementPositions.get(next).end;
+                        next++;
+                    } while (next < replacementPositions.size() && previousRegionEnd >= replacementPositions.get(next).start);
+
+                    // merge into a single replacement
+                    result.add(mergeLongestRegion(replacementPositions.subList(current, next)));
+                }
+
+                current = next;
+                next++;
+            }
+            return result;
+        }
+
+        /**
+         * Merge overlapping replacement regions into a single replacement.
+         * The name of the result comes from the longest replacement
+         *
+         * @param replacementPositions Must be sorted by start position
+         * @return Merged Replacement
+         */
+        static Replacement mergeLongestRegion(List<Replacement> replacementPositions) {
+            assert replacementPositions.size() > 1;
+
+            int longestIndex = 0;
+            int endPos = replacementPositions.get(0).end;
+            int maxLength = replacementPositions.get(0).length();
+
+            for (int i = 1; i < replacementPositions.size(); i++) {
+                if (replacementPositions.get(i).length() > maxLength) {
+                    maxLength = replacementPositions.get(i).length();
+                    longestIndex = i;
+                }
+                endPos = Math.max(endPos, replacementPositions.get(i).end);
+            }
+
+            return new Replacement(replacementPositions.get(0).start, endPos, replacementPositions.get(longestIndex).patternName);
+        }
+    }
+
+    public static final class Factory implements Processor.Factory {
+
+        private final MatcherWatchdog matcherWatchdog;
+
+        public Factory(MatcherWatchdog matcherWatchdog) {
+            this.matcherWatchdog = matcherWatchdog;
+        }
+
+        @Override
+        public RedactProcessor create(
+            Map<String, Processor.Factory> registry,
+            String processorTag,
+            String description,
+            Map<String, Object> config
+        ) throws Exception {
+            String matchField = ConfigurationUtils.readStringProperty(TYPE, processorTag, config, "field");
+            List<String> matchPatterns = ConfigurationUtils.readList(TYPE, processorTag, config, "patterns");
+            boolean ignoreMissing = ConfigurationUtils.readBooleanProperty(TYPE, processorTag, config, "ignore_missing", true);
+
+            String redactStart = ConfigurationUtils.readStringProperty(TYPE, processorTag, config, "prefix", DEFAULT_REDACTED_START);
+            String redactEnd = ConfigurationUtils.readStringProperty(TYPE, processorTag, config, "suffix", DEFAULT_REDACTED_END);
+
+            if (matchPatterns == null || matchPatterns.isEmpty()) {
+                throw newConfigurationException(TYPE, processorTag, "patterns", "List of patterns must not be empty");
+            }
+            Map<String, String> customPatternBank = ConfigurationUtils.readOptionalMap(TYPE, processorTag, config, "pattern_definitions");
+            Map<String, String> patternBank = new HashMap<>(Grok.getBuiltinPatterns(true));
+            if (customPatternBank != null) {
+                patternBank.putAll(customPatternBank);
+            }
+
+            try {
+                return new RedactProcessor(
+                    processorTag,
+                    description,
+                    patternBank,
+                    matchPatterns,
+                    matchField,
+                    ignoreMissing,
+                    redactStart,
+                    redactEnd,
+                    matcherWatchdog
+                );
+            } catch (Exception e) {
+                throw newConfigurationException(
+                    TYPE,
+                    processorTag,
+                    "patterns",
+                    "Invalid regex pattern found in: " + matchPatterns + ". " + e.getMessage()
+                );
+            }
+        }
+    }
+}

+ 63 - 0
modules/ingest-common/src/test/java/org/elasticsearch/ingest/common/RedactProcessorFactoryTests.java

@@ -0,0 +1,63 @@
+/*
+ * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
+ * or more contributor license agreements. Licensed under the Elastic License
+ * 2.0 and the Server Side Public License, v 1; you may not use this file except
+ * in compliance with, at your election, the Elastic License 2.0 or the Server
+ * Side Public License, v 1.
+ */
+
+package org.elasticsearch.ingest.common;
+
+import org.elasticsearch.ElasticsearchException;
+import org.elasticsearch.grok.MatcherWatchdog;
+import org.elasticsearch.test.ESTestCase;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.hamcrest.Matchers.containsString;
+import static org.hamcrest.Matchers.empty;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.hasSize;
+import static org.hamcrest.Matchers.not;
+
+public class RedactProcessorFactoryTests extends ESTestCase {
+
+    public void testPatternNotSet() {
+        RedactProcessor.Factory factory = new RedactProcessor.Factory(MatcherWatchdog.noop());
+
+        Map<String, Object> config = new HashMap<>();
+        config.put("field", "_field");
+        config.put("patterns", List.of());
+        ElasticsearchException e = expectThrows(ElasticsearchException.class, () -> factory.create(null, null, null, config));
+        assertThat(e.getMessage(), containsString("List of patterns must not be empty"));
+    }
+
+    public void testCreateWithCustomPatterns() throws Exception {
+        RedactProcessor.Factory factory = new RedactProcessor.Factory(MatcherWatchdog.noop());
+
+        Map<String, Object> config = new HashMap<>();
+        config.put("field", "_field");
+        config.put("patterns", List.of("%{MY_PATTERN:name}!"));
+        config.put("pattern_definitions", Map.of("MY_PATTERN", "foo"));
+        RedactProcessor processor = factory.create(null, null, null, config);
+        assertThat(processor.getGroks(), not(empty()));
+        assertThat(processor.getGroks().get(0).match("foo!"), equalTo(true));
+    }
+
+    public void testConfigKeysRemoved() throws Exception {
+        RedactProcessor.Factory factory = new RedactProcessor.Factory(MatcherWatchdog.noop());
+
+        Map<String, Object> config = new HashMap<>();
+        config.put("field", "_field");
+        config.put("patterns", List.of("%{MY_PATTERN:name}!"));
+        config.put("pattern_definitions", Map.of("MY_PATTERN", "foo"));
+        config.put("ignore_missing", true);
+        config.put("extra", "unused");
+
+        factory.create(null, null, null, config);
+        assertThat(config.entrySet(), hasSize(1));
+        assertEquals("unused", config.get("extra"));
+    }
+}

+ 362 - 0
modules/ingest-common/src/test/java/org/elasticsearch/ingest/common/RedactProcessorTests.java

@@ -0,0 +1,362 @@
+/*
+ * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
+ * or more contributor license agreements. Licensed under the Elastic License
+ * 2.0 and the Server Side Public License, v 1; you may not use this file except
+ * in compliance with, at your election, the Elastic License 2.0 or the Server
+ * Side Public License, v 1.
+ */
+
+package org.elasticsearch.ingest.common;
+
+import org.elasticsearch.grok.MatcherWatchdog;
+import org.elasticsearch.index.VersionType;
+import org.elasticsearch.ingest.IngestDocument;
+import org.elasticsearch.test.ESTestCase;
+
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.hamcrest.Matchers.contains;
+import static org.hamcrest.Matchers.containsString;
+import static org.hamcrest.Matchers.hasSize;
+import static org.hamcrest.Matchers.sameInstance;
+
+public class RedactProcessorTests extends ESTestCase {
+
+    public void testMatchRedact() throws Exception {
+        {
+            var config = new HashMap<String, Object>();
+            config.put("field", "to_redact");
+            config.put("patterns", List.of("%{EMAILADDRESS:EMAIL}"));
+            var processor = new RedactProcessor.Factory(MatcherWatchdog.noop()).create(null, "t", "d", config);
+            var groks = processor.getGroks();
+
+            {
+                String input = "thisisanemail@address.com will be redacted thisisdifferent@address.com";
+                var redacted = RedactProcessor.matchRedact(input, groks);
+                assertEquals("<EMAIL> will be redacted <EMAIL>", redacted);
+            }
+            {
+                String input = "This is ok nothing to redact";
+                var redacted = RedactProcessor.matchRedact(input, groks);
+                assertEquals(redacted, input);
+            }
+            {
+                String input = "thisisanemail@address.com will be redacted";
+                var redacted = RedactProcessor.matchRedact(input, groks);
+                assertEquals("<EMAIL> will be redacted", redacted);
+            }
+        }
+        {
+            var config = new HashMap<String, Object>();
+            config.put("field", "to_redact");
+            config.put("patterns", List.of("%{CREDIT_CARD:CREDIT_CARD}"));
+            config.put("pattern_definitions", Map.of("CREDIT_CARD", "\\b(?:\\d[ -]*?){13,16}\\b"));
+            var processor = new RedactProcessor.Factory(MatcherWatchdog.noop()).create(null, "t", "d", config);
+            var groks = processor.getGroks();
+
+            {
+                String input = "here is something that looks like a credit card number: 0001-0002-0003-0004";
+                var redacted = RedactProcessor.matchRedact(input, groks);
+                assertEquals("here is something that looks like a credit card number: <CREDIT_CARD>", redacted);
+            }
+            {
+                String input = "1001-1002-1003-1004 here is something that looks like a credit card number: 0001-0002-0003-0004";
+                var redacted = RedactProcessor.matchRedact(input, groks);
+                assertEquals("<CREDIT_CARD> here is something that looks like a credit card number: <CREDIT_CARD>", redacted);
+            }
+            {
+                String input = "1001-1002-1003-1004 some text in between 2001-1002-1003-1004 3001-1002-1003-1004 4001-1002-1003-1004";
+                var redacted = RedactProcessor.matchRedact(input, groks);
+                assertEquals("<CREDIT_CARD> some text in between <CREDIT_CARD> <CREDIT_CARD> <CREDIT_CARD>", redacted);
+            }
+            {
+                String input = "1001-1002-1003-1004 2001-1002-1003-1004 3001-1002-1003-1004 some 4001-1002-1003-1004"
+                    + " and lots more text here";
+                var redacted = RedactProcessor.matchRedact(input, groks);
+                assertEquals("<CREDIT_CARD> <CREDIT_CARD> <CREDIT_CARD> some <CREDIT_CARD> and lots more text here", redacted);
+            }
+        }
+        {
+            var config = new HashMap<String, Object>();
+            config.put("field", "to_redact");
+            config.put("patterns", List.of("%{CREDIT_CARD:CREDIT_CARD}"));
+            config.put("pattern_definitions", Map.of("CREDIT_CARD", "\\d{4}[ -]\\d{4}[ -]\\d{4}[ -]\\d{4}"));
+            var processor = new RedactProcessor.Factory(MatcherWatchdog.noop()).create(null, "t", "d", config);
+            var grok = processor.getGroks().get(0);
+
+            String input = "1001-1002-1003-1004 2001-1002-1003-1004 3001-1002-1003-1004 4001-1002-1003-1004";
+            var redacted = RedactProcessor.matchRedact(input, List.of(grok));
+            assertEquals("<CREDIT_CARD> <CREDIT_CARD> <CREDIT_CARD> <CREDIT_CARD>", redacted);
+        }
+    }
+
+    public void testMatchRedactMultipleGroks() throws Exception {
+        var config = new HashMap<String, Object>();
+        config.put("field", "to_redact");
+        config.put("patterns", List.of("%{EMAILADDRESS:EMAIL}", "%{IP:IP_ADDRESS}", "%{CREDIT_CARD:CREDIT_CARD}"));
+        config.put("pattern_definitions", Map.of("CREDIT_CARD", "\\d{4}[ -]\\d{4}[ -]\\d{4}[ -]\\d{4}"));
+        var processor = new RedactProcessor.Factory(MatcherWatchdog.noop()).create(null, "t", "d", config);
+        var groks = processor.getGroks();
+
+        {
+            String input = "thisisanemail@address.com will be redacted and this: 0001-0002-0003-0004 some other text";
+            var redacted = RedactProcessor.matchRedact(input, groks);
+            assertEquals("<EMAIL> will be redacted and this: <CREDIT_CARD> some other text", redacted);
+        }
+    }
+
+    public void testRedact() throws Exception {
+        var config = new HashMap<String, Object>();
+        config.put("field", "to_redact");
+        config.put("patterns", List.of("%{EMAILADDRESS:EMAIL}", "%{IP:IP_ADDRESS}", "%{CREDIT_CARD:CREDIT_CARD}"));
+        config.put("pattern_definitions", Map.of("CREDIT_CARD", "\\d{4}[ -]\\d{4}[ -]\\d{4}[ -]\\d{4}"));
+        var processor = new RedactProcessor.Factory(MatcherWatchdog.noop()).create(null, "t", "d", config);
+
+        {
+            var ingestDoc = createIngestDoc(Map.of("to_redact", "This is ok nothing to redact"));
+            var redacted = processor.execute(ingestDoc);
+            assertEquals(ingestDoc, redacted);
+        }
+        {
+            var ingestDoc = createIngestDoc(Map.of("to_redact", "thisisanemail@address.com will be redacted"));
+            var redacted = processor.execute(ingestDoc);
+            assertEquals("<EMAIL> will be redacted", redacted.getFieldValue("to_redact", String.class));
+        }
+        {
+            var ingestDoc = createIngestDoc(
+                Map.of("to_redact", "here is something that looks like a credit card number: 0001-0002-0003-0004")
+            );
+            var redacted = processor.execute(ingestDoc);
+            assertEquals(
+                "here is something that looks like a credit card number: <CREDIT_CARD>",
+                redacted.getFieldValue("to_redact", String.class)
+            );
+        }
+    }
+
+    public void testRedactWithPatternNamesRedacted() throws Exception {
+        var config = new HashMap<String, Object>();
+        config.put("field", "to_redact");
+        config.put("patterns", List.of("%{EMAILADDRESS:REDACTED}", "%{IP:REDACTED}", "%{CREDIT_CARD:REDACTED}"));
+        config.put("pattern_definitions", Map.of("CREDIT_CARD", "\\d{4}[ -]\\d{4}[ -]\\d{4}[ -]\\d{4}"));
+        var processor = new RedactProcessor.Factory(MatcherWatchdog.noop()).create(null, "t", "d", config);
+
+        {
+            var ingestDoc = createIngestDoc(Map.of("to_redact", "look a credit card number! 0001-0002-0003-0004 from david@email.com"));
+            var redacted = processor.execute(ingestDoc);
+            assertEquals("look a credit card number! <REDACTED> from <REDACTED>", redacted.getFieldValue("to_redact", String.class));
+        }
+    }
+
+    public void testDifferentStartAndEnd() throws Exception {
+        {
+            var config = new HashMap<String, Object>();
+            config.put("field", "to_redact");
+            config.put("patterns", List.of("%{EMAILADDRESS:EMAIL}", "%{IP:IP_ADDRESS}"));
+            config.put("prefix", "?--");
+            config.put("suffix", "}");
+
+            var processor = new RedactProcessor.Factory(MatcherWatchdog.noop()).create(null, "t", "d", config);
+            var ingestDoc = createIngestDoc(Map.of("to_redact", "0.0.0.1 will be redacted"));
+            var redacted = processor.execute(ingestDoc);
+            assertEquals("?--IP_ADDRESS} will be redacted", redacted.getFieldValue("to_redact", String.class));
+        }
+        {
+            var config = new HashMap<String, Object>();
+            config.put("field", "to_redact");
+            config.put("patterns", List.of("%{IP:IP_ADDRESS}"));
+            config.put("prefix", "?--");
+
+            var processor = new RedactProcessor.Factory(MatcherWatchdog.noop()).create(null, "t", "d", config);
+            var ingestDoc = createIngestDoc(Map.of("to_redact", "0.0.0.1 will be redacted"));
+            var redacted = processor.execute(ingestDoc);
+            assertEquals("?--IP_ADDRESS> will be redacted", redacted.getFieldValue("to_redact", String.class));
+        }
+        {
+            var config = new HashMap<String, Object>();
+            config.put("field", "to_redact");
+            config.put("patterns", List.of("%{IP:IP_ADDRESS}"));
+            config.put("suffix", "++");
+
+            var processor = new RedactProcessor.Factory(MatcherWatchdog.noop()).create(null, "t", "d", config);
+            var ingestDoc = createIngestDoc(Map.of("to_redact", "0.0.0.1 will be redacted"));
+            var redacted = processor.execute(ingestDoc);
+            assertEquals("<IP_ADDRESS++ will be redacted", redacted.getFieldValue("to_redact", String.class));
+        }
+    }
+
+    public void testIgnoreMissing() throws Exception {
+        {
+            var config = new HashMap<String, Object>();
+            config.put("field", "to_redact");
+            config.put("patterns", List.of("foo"));
+            var processor = new RedactProcessor.Factory(MatcherWatchdog.noop()).create(null, "t", "d", config);
+            var ingestDoc = createIngestDoc(Map.of("not_the_field", "fieldValue"));
+            var processed = processor.execute(ingestDoc);
+            assertThat(ingestDoc, sameInstance(processed));
+            assertEquals(ingestDoc, processed);
+        }
+        {
+            var config = new HashMap<String, Object>();
+            config.put("field", "to_redact");
+            config.put("patterns", List.of("foo"));
+            config.put("ignore_missing", false);   // this time the missing field should error
+
+            var processor = new RedactProcessor.Factory(MatcherWatchdog.noop()).create(null, "t", "d", config);
+            var ingestDoc = createIngestDoc(Map.of("not_the_field", "fieldValue"));
+            IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> processor.execute(ingestDoc));
+            assertThat(e.getMessage(), containsString("field [to_redact] is null or missing"));
+        }
+    }
+
+    public void testMergeLongestRegion() {
+        var r = List.of(
+            new RedactProcessor.RegionTrackingMatchExtractor.Replacement(10, 20, "first"),
+            new RedactProcessor.RegionTrackingMatchExtractor.Replacement(15, 28, "longest"),
+            new RedactProcessor.RegionTrackingMatchExtractor.Replacement(22, 29, "third")
+        );
+
+        var merged = RedactProcessor.RegionTrackingMatchExtractor.mergeLongestRegion(r);
+        assertEquals("longest", merged.patternName);
+        assertEquals(10, merged.start);
+        assertEquals(29, merged.end);
+    }
+
+    public void testMergeLongestRegion_smallRegionSubsumed() {
+        {
+            var r = List.of(
+                new RedactProcessor.RegionTrackingMatchExtractor.Replacement(10, 50, "longest"),
+                new RedactProcessor.RegionTrackingMatchExtractor.Replacement(15, 25, "subsumed")
+            );
+
+            var merged = RedactProcessor.RegionTrackingMatchExtractor.mergeLongestRegion(r);
+            assertEquals("longest", merged.patternName);
+            assertEquals(10, merged.start);
+            assertEquals(50, merged.end);
+        }
+        {
+            var r = List.of(
+                new RedactProcessor.RegionTrackingMatchExtractor.Replacement(10, 50, "longest"),
+                new RedactProcessor.RegionTrackingMatchExtractor.Replacement(15, 25, "subsumed"),
+                new RedactProcessor.RegionTrackingMatchExtractor.Replacement(44, 60, "third")
+            );
+
+            var merged = RedactProcessor.RegionTrackingMatchExtractor.mergeLongestRegion(r);
+            assertEquals("longest", merged.patternName);
+            assertEquals(10, merged.start);
+            assertEquals(60, merged.end);
+        }
+    }
+
+    public void testMergeOverlappingReplacements_sortedByStartPositionNoOverlaps() {
+        var a1 = new RedactProcessor.RegionTrackingMatchExtractor.Replacement(35, 40, "A");
+        var b1 = new RedactProcessor.RegionTrackingMatchExtractor.Replacement(5, 12, "B");
+        var b2 = new RedactProcessor.RegionTrackingMatchExtractor.Replacement(30, 34, "B");
+        var c1 = new RedactProcessor.RegionTrackingMatchExtractor.Replacement(21, 29, "C");
+
+        var merged = RedactProcessor.RegionTrackingMatchExtractor.mergeOverlappingReplacements(Arrays.asList(a1, b1, b2, c1));
+        assertThat(merged, contains(b1, c1, b2, a1));
+    }
+
+    public void testMergeOverlappingReplacements_singleItem() {
+        var l = List.of(new RedactProcessor.RegionTrackingMatchExtractor.Replacement(35, 40, "A"));
+        var merged = RedactProcessor.RegionTrackingMatchExtractor.mergeOverlappingReplacements(l);
+        assertThat(merged, sameInstance(l));
+    }
+
+    public void testMergeOverlappingReplacements_transitiveOverlaps() {
+        {
+            var a1 = new RedactProcessor.RegionTrackingMatchExtractor.Replacement(35, 40, "A");
+            var b1 = new RedactProcessor.RegionTrackingMatchExtractor.Replacement(5, 10, "B");
+            var b2 = new RedactProcessor.RegionTrackingMatchExtractor.Replacement(10, 15, "B");
+            var c1 = new RedactProcessor.RegionTrackingMatchExtractor.Replacement(14, 29, "C");
+
+            // b1, b2 and c1 overlap and should be merged into a single replacement
+            var merged = RedactProcessor.RegionTrackingMatchExtractor.mergeOverlappingReplacements(Arrays.asList(a1, b1, b2, c1));
+            assertThat(merged, hasSize(2));
+            var mergedRegion = merged.get(0);
+            assertEquals("C", mergedRegion.patternName);
+            assertEquals(5, mergedRegion.start);
+            assertEquals(29, mergedRegion.end);
+            assertEquals(a1, merged.get(1));
+        }
+        {
+            var a1 = new RedactProcessor.RegionTrackingMatchExtractor.Replacement(20, 28, "A");
+            var a2 = new RedactProcessor.RegionTrackingMatchExtractor.Replacement(50, 60, "A");
+            var b1 = new RedactProcessor.RegionTrackingMatchExtractor.Replacement(30, 39, "B");
+            var b2 = new RedactProcessor.RegionTrackingMatchExtractor.Replacement(59, 65, "B");
+            var c1 = new RedactProcessor.RegionTrackingMatchExtractor.Replacement(14, 18, "C");
+
+            // a2 and b2 overlap
+            var merged = RedactProcessor.RegionTrackingMatchExtractor.mergeOverlappingReplacements(Arrays.asList(a1, a2, b1, b2, c1));
+            assertThat(merged, hasSize(4));
+            assertEquals(c1, merged.get(0));
+            assertEquals(a1, merged.get(1));
+            assertEquals(b1, merged.get(2));
+            var mergedRegion = merged.get(3);
+            assertEquals("A", mergedRegion.patternName);
+            assertEquals(50, mergedRegion.start);
+            assertEquals(65, mergedRegion.end);
+        }
+        {
+            var a1 = new RedactProcessor.RegionTrackingMatchExtractor.Replacement(20, 28, "A");
+            var a2 = new RedactProcessor.RegionTrackingMatchExtractor.Replacement(50, 60, "A");
+            var b1 = new RedactProcessor.RegionTrackingMatchExtractor.Replacement(14, 19, "B");
+            var b2 = new RedactProcessor.RegionTrackingMatchExtractor.Replacement(30, 39, "B");
+            var c1 = new RedactProcessor.RegionTrackingMatchExtractor.Replacement(39, 49, "C");
+
+            var merged = RedactProcessor.RegionTrackingMatchExtractor.mergeOverlappingReplacements(Arrays.asList(a1, a2, b1, b2, c1));
+            assertThat(merged, hasSize(4));
+            assertEquals(b1, merged.get(0));
+            assertEquals(a1, merged.get(1));
+            var mergedRegion = merged.get(2);
+            assertEquals("C", mergedRegion.patternName);
+            assertEquals(30, mergedRegion.start);
+            assertEquals(49, mergedRegion.end);
+            assertEquals(a2, merged.get(3));
+        }
+        {
+            var a1 = new RedactProcessor.RegionTrackingMatchExtractor.Replacement(20, 28, "A");
+            var a2 = new RedactProcessor.RegionTrackingMatchExtractor.Replacement(50, 60, "A");
+            var b1 = new RedactProcessor.RegionTrackingMatchExtractor.Replacement(22, 26, "B");
+
+            // b1 subsumed by a1
+            var merged = RedactProcessor.RegionTrackingMatchExtractor.mergeOverlappingReplacements(Arrays.asList(a1, a2, b1));
+            assertThat(merged, hasSize(2));
+            var mergedRegion = merged.get(0);
+            assertEquals("A", mergedRegion.patternName);
+            assertEquals(20, mergedRegion.start);
+            assertEquals(28, mergedRegion.end);
+            assertEquals(a2, merged.get(1));
+        }
+        {
+            var a1 = new RedactProcessor.RegionTrackingMatchExtractor.Replacement(20, 28, "A");
+            var a2 = new RedactProcessor.RegionTrackingMatchExtractor.Replacement(50, 60, "A");
+            var b1 = new RedactProcessor.RegionTrackingMatchExtractor.Replacement(14, 21, "B");
+            var b2 = new RedactProcessor.RegionTrackingMatchExtractor.Replacement(30, 36, "B");
+            var c1 = new RedactProcessor.RegionTrackingMatchExtractor.Replacement(44, 51, "C");
+            var c2 = new RedactProcessor.RegionTrackingMatchExtractor.Replacement(62, 70, "C");
+
+            // a1 and b1 merged. c1 and a2 merged
+            var merged = RedactProcessor.RegionTrackingMatchExtractor.mergeOverlappingReplacements(Arrays.asList(a1, a2, b1, b2, c1, c2));
+            assertThat(merged, hasSize(4));
+            var mergedRegion = merged.get(0);
+            assertEquals("A", mergedRegion.patternName);
+            assertEquals(14, mergedRegion.start);
+            assertEquals(28, mergedRegion.end);
+            assertEquals(b2, merged.get(1));
+            mergedRegion = merged.get(2);
+            assertEquals("A", mergedRegion.patternName);
+            assertEquals(44, mergedRegion.start);
+            assertEquals(60, mergedRegion.end);
+            assertEquals(c2, merged.get(3));
+        }
+    }
+
+    private IngestDocument createIngestDoc(Map<String, Object> source) {
+        return new IngestDocument("index", "id", 0L, "routing", VersionType.INTERNAL, source);
+    }
+}

+ 98 - 0
modules/ingest-common/src/yamlRestTest/resources/rest-api-spec/test/ingest/300_redact_processor.yml

@@ -0,0 +1,98 @@
+---
+"Test put redact pipeline":
+  - do:
+      ingest.put_pipeline:
+        id: "pipeline-using-a-redact-processor"
+        body: >
+          {
+            "processors": [
+              {
+                "redact": {
+                  "field": "to_redact",
+                  "patterns": ["%{EMAILADDRESS:EMAIL}", "%{IP:IP_ADDRESS}", "%{CREDIT_CARD:CREDIT_CARD}"],
+                  "pattern_definitions": {
+                    "CREDIT_CARD": "\\d{4}[ -]\\d{4}[ -]\\d{4}[ -]\\d{4}"
+                  }
+                }
+              }
+            ]
+          }
+  - match: { acknowledged: true }
+
+  - do:
+      index:
+        index: test
+        id: "1"
+        pipeline: "pipeline-using-a-redact-processor"
+        body: {to_redact: "0.0.0.1 is my secret IP to redact"}
+
+  - do:
+      get:
+        index: test
+        id: "1"
+  - match: { _source.to_redact: "<IP_ADDRESS> is my secret IP to redact" }
+
+---
+"Test redact":
+  - do:
+      ingest.simulate:
+        body: >
+          {
+            "pipeline": {
+              "processors": [
+                {
+                  "redact": {
+                    "field": "to_redact",
+                    "patterns": ["%{EMAILADDRESS:EMAIL}", "%{IP:IP_ADDRESS}"]
+                  }
+                }
+              ]
+            },
+            "docs": [{"_source": {"to_redact": "this-email@address.com will be redacted"}}]
+          }
+  - length: { docs: 1 }
+  - match: { docs.0.doc._source.to_redact: "<EMAIL> will be redacted" }
+
+---
+"Test ignore missing":
+  - do:
+      ingest.simulate:
+        body: >
+          {
+            "pipeline": {
+              "processors": [
+                {
+                  "redact": {
+                    "field": "to_redact",
+                    "ignore_missing": false,
+                    "patterns": ["%{EMAILADDRESS:EMAIL}", "%{IP:IP_ADDRESS}"]
+                  }
+                }
+              ]
+            },
+            "docs": [{"_source": {"wrong_field": "will error"}}]
+          }
+  - match: { docs.0.error.reason: "field [to_redact] is null or missing" }
+
+---
+"Test custom start and end tokens":
+  - do:
+      ingest.simulate:
+        body: >
+          {
+            "pipeline": {
+              "processors": [
+                {
+                  "redact": {
+                    "prefix": "==*",
+                    "suffix": "*==",
+                    "field": "to_redact",
+                    "patterns": ["%{EMAILADDRESS:EMAIL}", "%{IP:IP_ADDRESS}"]
+                  }
+                }
+              ]
+            },
+            "docs": [{"_source": {"to_redact": "this-email@address.com will be redacted"}}]
+          }
+  - length: { docs: 1 }
+  - match: { docs.0.doc._source.to_redact: "==*EMAIL*== will be redacted" }