You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@logging.apache.org by vy...@apache.org on 2022/08/17 09:30:55 UTC

[logging-log4j2] 01/03: LOG4J2-3556 Make JsonTemplateLayout stack trace truncation operate for each label block.

This is an automated email from the ASF dual-hosted git repository.

vy pushed a commit to branch LOG4J2-3556
in repository https://gitbox.apache.org/repos/asf/logging-log4j2.git

commit 498fa4f12b7c3a2dbba19d5c86101f31ecab1d3b
Author: Volkan Yazıcı <vo...@yazi.ci>
AuthorDate: Mon Aug 15 23:01:45 2022 +0200

    LOG4J2-3556 Make JsonTemplateLayout stack trace truncation operate for each label block.
---
 .../template/json/resolver/ExceptionResolver.java  |   9 +-
 .../json/resolver/StackTraceStringResolver.java    | 242 ++++++++++--
 .../template/json/util/CharSequencePointer.java    | 106 +++++
 .../json/util/TruncatingBufferedPrintWriter.java   |  17 +-
 .../json/util/TruncatingBufferedWriter.java        |  50 +--
 .../resolver/StackTraceStringResolverTest.java     | 437 +++++++++++++++++++++
 .../json/util/CharSequencePointerTest.java         | 121 ++++++
 .../json/util/TruncatingBufferedWriterTest.java    |  63 ++-
 src/changes/changes.xml                            |   3 +
 .../asciidoc/manual/json-template-layout.adoc.vm   |  10 +-
 10 files changed, 985 insertions(+), 73 deletions(-)

diff --git a/log4j-layout-template-json/src/main/java/org/apache/logging/log4j/layout/template/json/resolver/ExceptionResolver.java b/log4j-layout-template-json/src/main/java/org/apache/logging/log4j/layout/template/json/resolver/ExceptionResolver.java
index e2a538a7a5..5c31e00b33 100644
--- a/log4j-layout-template-json/src/main/java/org/apache/logging/log4j/layout/template/json/resolver/ExceptionResolver.java
+++ b/log4j-layout-template-json/src/main/java/org/apache/logging/log4j/layout/template/json/resolver/ExceptionResolver.java
@@ -71,8 +71,13 @@ import java.util.regex.PatternSyntaxException;
  * first.
  * <p>
  * If a stringified stack trace truncation takes place, it will be indicated
- * with <tt>suffix</tt>, which by default is set to the configured
+ * with a <tt>suffix</tt>, which by default is set to the configured
  * <tt>truncatedStringSuffix</tt> in the layout, unless explicitly provided.
+ * Every truncation suffix is prefixed with a newline.
+ * <p>
+ * Stringified stack trace truncation operates in <tt>Caused by:</tt> and
+ * <tt>Suppressed:</tt> label blocks. That is, matchers are executed against
+ * each label in isolation.
  * <p>
  * <tt>elementTemplate</tt> is an object describing the template to be used
  * while resolving the {@link StackTraceElement} array. If <tt>stringified</tt>
@@ -138,7 +143,7 @@ import java.util.regex.PatternSyntaxException;
  *   "stackTrace": {
  *     "stringified": {
  *       "truncation": {
- *         "suffix": ">",
+ *         "suffix": "... [truncated]",
  *         "pointMatcherStrings": ["at javax.servlet.http.HttpServlet.service"]
  *       }
  *     }
diff --git a/log4j-layout-template-json/src/main/java/org/apache/logging/log4j/layout/template/json/resolver/StackTraceStringResolver.java b/log4j-layout-template-json/src/main/java/org/apache/logging/log4j/layout/template/json/resolver/StackTraceStringResolver.java
index 0dee8d08e8..4e72d1a718 100644
--- a/log4j-layout-template-json/src/main/java/org/apache/logging/log4j/layout/template/json/resolver/StackTraceStringResolver.java
+++ b/log4j-layout-template-json/src/main/java/org/apache/logging/log4j/layout/template/json/resolver/StackTraceStringResolver.java
@@ -16,9 +16,7 @@
  */
 package org.apache.logging.log4j.layout.template.json.resolver;
 
-import org.apache.logging.log4j.layout.template.json.util.TruncatingBufferedPrintWriter;
-import org.apache.logging.log4j.layout.template.json.util.JsonWriter;
-import org.apache.logging.log4j.layout.template.json.util.Recycler;
+import org.apache.logging.log4j.layout.template.json.util.*;
 
 import java.util.List;
 import java.util.function.Supplier;
@@ -31,7 +29,11 @@ import java.util.stream.Collectors;
  */
 final class StackTraceStringResolver implements StackTraceResolver {
 
-    private final Recycler<TruncatingBufferedPrintWriter> writerRecycler;
+    private final Recycler<TruncatingBufferedPrintWriter> srcWriterRecycler;
+
+    private final Recycler<TruncatingBufferedPrintWriter> dstWriterRecycler;
+
+    private final Recycler<CharSequencePointer> sequencePointerRecycler;
 
     private final boolean truncationEnabled;
 
@@ -49,9 +51,15 @@ final class StackTraceStringResolver implements StackTraceResolver {
         final Supplier<TruncatingBufferedPrintWriter> writerSupplier =
                 () -> TruncatingBufferedPrintWriter.ofCapacity(
                         context.getMaxStringByteCount());
-        this.writerRecycler = context
-                .getRecyclerFactory()
-                .create(writerSupplier, TruncatingBufferedPrintWriter::close);
+        final RecyclerFactory recyclerFactory = context.getRecyclerFactory();
+        this.srcWriterRecycler =
+                recyclerFactory.create(
+                        writerSupplier, TruncatingBufferedPrintWriter::close);
+        this.dstWriterRecycler =
+                recyclerFactory.create(
+                        writerSupplier, TruncatingBufferedPrintWriter::close);
+        this.sequencePointerRecycler =
+                recyclerFactory.create(CharSequencePointer::new);
         this.truncationEnabled =
                 !truncationPointMatcherStrings.isEmpty() ||
                         !truncationPointMatcherRegexes.isEmpty();
@@ -75,56 +83,236 @@ final class StackTraceStringResolver implements StackTraceResolver {
     public void resolve(
             final Throwable throwable,
             final JsonWriter jsonWriter) {
-        final TruncatingBufferedPrintWriter writer = writerRecycler.acquire();
+        final TruncatingBufferedPrintWriter srcWriter = srcWriterRecycler.acquire();
         try {
-            throwable.printStackTrace(writer);
-            truncate(writer);
-            jsonWriter.writeString(writer.buffer(), 0, writer.position());
+            throwable.printStackTrace(srcWriter);
+            final TruncatingBufferedPrintWriter dstWriter = truncate(srcWriter);
+            jsonWriter.writeString(dstWriter);
         } finally {
-            writerRecycler.release(writer);
+            srcWriterRecycler.release(srcWriter);
         }
     }
 
-    private void truncate(final TruncatingBufferedPrintWriter writer) {
+    private TruncatingBufferedPrintWriter truncate(
+            final TruncatingBufferedPrintWriter srcWriter) {
 
         // Short-circuit if truncation is not enabled.
         if (!truncationEnabled) {
-            return;
+            return srcWriter;
+        }
+
+        // Allocate temporary buffers and truncate the input.
+        final TruncatingBufferedPrintWriter dstWriter =
+                dstWriterRecycler.acquire();
+        try {
+            final CharSequencePointer sequencePointer =
+                    sequencePointerRecycler.acquire();
+            try {
+                truncate(srcWriter, dstWriter, sequencePointer);
+            } finally {
+                sequencePointerRecycler.release(sequencePointer);
+            }
+        } finally {
+            dstWriterRecycler.release(dstWriter);
+        }
+        return dstWriter;
+
+    }
+
+    private void truncate(
+            final TruncatingBufferedPrintWriter srcWriter,
+            final TruncatingBufferedPrintWriter dstWriter,
+            final CharSequencePointer sequencePointer) {
+        int startIndex = 0;
+        for (;;) {
+
+            // Find the next label start, if present.
+            final int labeledLineStartIndex =
+                    findLabeledLineStartIndex(
+                            srcWriter, startIndex, srcWriter.length());
+            final int endIndex = labeledLineStartIndex >= 0
+                    ? labeledLineStartIndex
+                    : srcWriter.length();
+
+            // Copy up to the truncation point, if it matches.
+            final int truncationPointIndex = findTruncationPointIndex(
+                    srcWriter, startIndex, endIndex, sequencePointer);
+            if (truncationPointIndex > 0) {
+                dstWriter.append(srcWriter, startIndex, truncationPointIndex);
+                dstWriter.append(System.lineSeparator());
+                dstWriter.append(truncationSuffix);
+            }
+
+            // Otherwise, copy the entire labeled block.
+            else {
+                dstWriter.append(srcWriter, startIndex, endIndex);
+            }
+
+            // Copy the label to avoid stepping over it again.
+            if (labeledLineStartIndex > 0) {
+                dstWriter.append(System.lineSeparator());
+                startIndex = labeledLineStartIndex;
+                for (;;) {
+                    final char c = srcWriter.charAt(startIndex++);
+                    dstWriter.append(c);
+                    if (c == ':') {
+                        break;
+                    }
+                }
+            }
+
+            // Otherwise, the source is exhausted, stop.
+            else {
+                break;
+            }
+
         }
+    }
+
+    private int findTruncationPointIndex(
+            final TruncatingBufferedPrintWriter writer,
+            final int startIndex,
+            final int endIndex,
+            final CharSequencePointer sequencePointer) {
 
         // Check for string matches.
         // noinspection ForLoopReplaceableByForEach (avoid iterator allocation)
         for (int i = 0; i < truncationPointMatcherStrings.size(); i++) {
             final String matcher = truncationPointMatcherStrings.get(i);
-            final int matchIndex = writer.indexOf(matcher);
+            final int matchIndex = findMatchingIndex(
+                    matcher, writer, startIndex, endIndex);
             if (matchIndex > 0) {
-                final int truncationPointIndex = matchIndex + matcher.length();
-                truncate(writer, truncationPointIndex);
-                return;
+                // No need for `Math.addExact()`, since we have a match:
+                return matchIndex + matcher.length();
             }
         }
 
         // Check for regex matches.
+        CharSequence sequence;
+        if (startIndex == 0 && endIndex == writer.length()) {
+            sequence = writer;
+        } else {
+            sequencePointer.reset(writer, startIndex, writer.length());
+            sequence = sequencePointer;
+        }
         // noinspection ForLoopReplaceableByForEach (avoid iterator allocation)
         for (int i = 0; i < groupedTruncationPointMatcherRegexes.size(); i++) {
             final Pattern pattern = groupedTruncationPointMatcherRegexes.get(i);
-            final Matcher matcher = pattern.matcher(writer);
+            final Matcher matcher = pattern.matcher(sequence);
             final boolean matched = matcher.matches();
             if (matched) {
                 final int lastGroup = matcher.groupCount();
-                final int truncationPointIndex = matcher.start(lastGroup);
-                truncate(writer, truncationPointIndex);
-                return;
+                return matcher.start(lastGroup);
             }
         }
 
+        // No matches.
+        return -1;
+
     }
 
-    private void truncate(
-            final TruncatingBufferedPrintWriter writer,
-            final int index) {
-        writer.position(index);
-        writer.print(truncationSuffix);
+    private static int findLabeledLineStartIndex(
+            final CharSequence buffer,
+            final int startIndex,
+            final int endIndex) {
+        // Note that the index arithmetic in this method is not guarded.
+        // That is, there are no `Math.addExact()` or `Math.subtractExact()` usages.
+        // Since we know a priori that we are already operating within buffer limits.
+        for (int bufferIndex = startIndex; bufferIndex < endIndex;) {
+
+            // Find the next line start, if exists.
+            final int lineStartIndex = findLineStartIndex(buffer, bufferIndex, endIndex);
+            if (lineStartIndex < 0) {
+                break;
+            }
+            bufferIndex = lineStartIndex;
+
+            // Skip tabs.
+            while (bufferIndex < endIndex && '\t' == buffer.charAt(bufferIndex)) {
+                bufferIndex++;
+            }
+
+            // Search for the `Caused by: ` occurrence.
+            if (bufferIndex < (endIndex - 11) &&
+                    buffer.charAt(bufferIndex) == 'C' &&
+                    buffer.charAt(bufferIndex + 1) == 'a' &&
+                    buffer.charAt(bufferIndex + 2) == 'u' &&
+                    buffer.charAt(bufferIndex + 3) == 's' &&
+                    buffer.charAt(bufferIndex + 4) == 'e' &&
+                    buffer.charAt(bufferIndex + 5) == 'd' &&
+                    buffer.charAt(bufferIndex + 6) == ' ' &&
+                    buffer.charAt(bufferIndex + 7) == 'b' &&
+                    buffer.charAt(bufferIndex + 8) == 'y' &&
+                    buffer.charAt(bufferIndex + 9) == ':' &&
+                    buffer.charAt(bufferIndex + 10) == ' ') {
+                return lineStartIndex;
+            }
+
+            // Search for the `Suppressed: ` occurrence.
+            else if (bufferIndex < (endIndex - 12) &&
+                    buffer.charAt(bufferIndex) == 'S' &&
+                    buffer.charAt(bufferIndex + 1) == 'u' &&
+                    buffer.charAt(bufferIndex + 2) == 'p' &&
+                    buffer.charAt(bufferIndex + 3) == 'p' &&
+                    buffer.charAt(bufferIndex + 4) == 'r' &&
+                    buffer.charAt(bufferIndex + 5) == 'e' &&
+                    buffer.charAt(bufferIndex + 6) == 's' &&
+                    buffer.charAt(bufferIndex + 7) == 's' &&
+                    buffer.charAt(bufferIndex + 8) == 'e' &&
+                    buffer.charAt(bufferIndex + 9) == 'd' &&
+                    buffer.charAt(bufferIndex + 10) == ':' &&
+                    buffer.charAt(bufferIndex + 11) == ' ') {
+                return lineStartIndex;
+            }
+
+        }
+        return -1;
+    }
+
+    private static int findLineStartIndex(
+            final CharSequence buffer,
+            final int startIndex,
+            final int endIndex) {
+        char prevChar = '-';
+        for (int i = startIndex; i <= endIndex; i++) {
+            if (prevChar == '\n') {
+                return i;
+            }
+            prevChar = buffer.charAt(i);
+        }
+        return -1;
+    }
+
+    private static int findMatchingIndex(
+            final CharSequence matcher,
+            final CharSequence buffer,
+            final int bufferStartIndex,
+            final int bufferEndIndex) {
+
+        // Note that the index arithmetic in this method is not guarded.
+        // That is, there are no `Math.addExact()` or `Math.subtractExact()` usages.
+        // Since we know a priori that we are already operating within buffer limits.
+
+        // While searching for an input of length `n`, no need to traverse the last `n-1` characters.
+        final int effectiveBufferEndIndex = bufferEndIndex - matcher.length() + 1;
+
+        // Perform the search.
+        for (int bufferIndex = bufferStartIndex; bufferIndex <= effectiveBufferEndIndex; bufferIndex++) {
+            boolean found = true;
+            for (int matcherIndex = 0; matcherIndex < matcher.length(); matcherIndex++) {
+                final char matcherChar = matcher.charAt(matcherIndex);
+                final char bufferChar = buffer.charAt(bufferIndex + matcherIndex);
+                if (matcherChar != bufferChar) {
+                    found = false;
+                    break;
+                }
+            }
+            if (found) {
+                return bufferIndex;
+            }
+        }
+        return -1;
+
     }
 
 }
diff --git a/log4j-layout-template-json/src/main/java/org/apache/logging/log4j/layout/template/json/util/CharSequencePointer.java b/log4j-layout-template-json/src/main/java/org/apache/logging/log4j/layout/template/json/util/CharSequencePointer.java
new file mode 100644
index 0000000000..0495e7c7c4
--- /dev/null
+++ b/log4j-layout-template-json/src/main/java/org/apache/logging/log4j/layout/template/json/util/CharSequencePointer.java
@@ -0,0 +1,106 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF 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.apache.logging.log4j.layout.template.json.util;
+
+import java.util.Objects;
+import java.util.stream.IntStream;
+
+/**
+ * A {@link CharSequence} wrapper that allows mutation of the pointed delegate sequence.
+ */
+public final class CharSequencePointer implements CharSequence {
+
+    private CharSequence delegate;
+
+    private int startIndex;
+
+    private int length = -1;
+
+    public void reset(
+            final CharSequence delegate,
+            final int startIndex,
+            final int endIndex) {
+
+        // Check & set the delegate.
+        Objects.requireNonNull(delegate, "delegate");
+        this.delegate = delegate;
+
+        // Check & set the start.
+        if (startIndex < 0) {
+            throw new IndexOutOfBoundsException("invalid start: " + startIndex);
+        }
+
+        // Check & set length.
+        if (endIndex > delegate.length()) {
+            throw new IndexOutOfBoundsException("invalid end: " + endIndex);
+        }
+        this.length = Math.subtractExact(endIndex, startIndex);
+        if (length < 0) {
+            throw new IndexOutOfBoundsException("invalid length: " + length);
+        }
+
+        // Set fields.
+        this.delegate = delegate;
+        this.startIndex = startIndex;
+
+    }
+
+    @Override
+    public int length() {
+        requireReset();
+        return length;
+    }
+
+    @Override
+    public char charAt(final int startIndex) {
+        requireReset();
+        final int delegateStartIndex = Math.addExact(this.startIndex, startIndex);
+        return delegate.charAt(delegateStartIndex);
+    }
+
+    @Override
+    public CharSequence subSequence(final int startIndex, final int endIndex) {
+        throw new UnsupportedOperationException(
+                "operation requires allocation, contradicting with the purpose of the class");
+    }
+
+    @Override
+    public IntStream chars() {
+        throw new UnsupportedOperationException(
+                "operation requires allocation, contradicting with the purpose of the class");
+    }
+
+    @Override
+    public IntStream codePoints() {
+        throw new UnsupportedOperationException(
+                "operation requires allocation, contradicting with the purpose of the class");
+    }
+
+    @Override
+    public String toString() {
+        requireReset();
+        final int endIndex = Math.addExact(startIndex, length);
+        return delegate.toString().substring(startIndex, endIndex);
+    }
+
+    private void requireReset() {
+        if (length < 0) {
+            throw new IllegalStateException("pointer must be reset first");
+        }
+    }
+
+}
diff --git a/log4j-layout-template-json/src/main/java/org/apache/logging/log4j/layout/template/json/util/TruncatingBufferedPrintWriter.java b/log4j-layout-template-json/src/main/java/org/apache/logging/log4j/layout/template/json/util/TruncatingBufferedPrintWriter.java
index 7e9aa3cc82..7f30ab372d 100644
--- a/log4j-layout-template-json/src/main/java/org/apache/logging/log4j/layout/template/json/util/TruncatingBufferedPrintWriter.java
+++ b/log4j-layout-template-json/src/main/java/org/apache/logging/log4j/layout/template/json/util/TruncatingBufferedPrintWriter.java
@@ -59,11 +59,6 @@ public final class TruncatingBufferedPrintWriter
         return writer.truncated();
     }
 
-    public int indexOf(final CharSequence seq) {
-        Objects.requireNonNull(seq, "seq");
-        return writer.indexOf(seq);
-    }
-
     @Override
     public int length() {
         return writer.length();
@@ -74,6 +69,18 @@ public final class TruncatingBufferedPrintWriter
         return writer.charAt(index);
     }
 
+    @Override
+    public PrintWriter append(final CharSequence seq) {
+        writer.append(seq);
+        return this;
+    }
+
+    @Override
+    public PrintWriter append(final CharSequence seq, final int startIndex, final int endIndex) {
+        writer.append(seq, startIndex, endIndex);
+        return this;
+    }
+
     @Override
     public CharSequence subSequence(final int startIndex, final int endIndex) {
         return writer.subSequence(startIndex, endIndex);
diff --git a/log4j-layout-template-json/src/main/java/org/apache/logging/log4j/layout/template/json/util/TruncatingBufferedWriter.java b/log4j-layout-template-json/src/main/java/org/apache/logging/log4j/layout/template/json/util/TruncatingBufferedWriter.java
index 1b88f121d4..62757a3f37 100644
--- a/log4j-layout-template-json/src/main/java/org/apache/logging/log4j/layout/template/json/util/TruncatingBufferedWriter.java
+++ b/log4j-layout-template-json/src/main/java/org/apache/logging/log4j/layout/template/json/util/TruncatingBufferedWriter.java
@@ -18,6 +18,7 @@ package org.apache.logging.log4j.layout.template.json.util;
 
 import java.io.Writer;
 import java.util.Objects;
+import java.util.stream.IntStream;
 
 final class TruncatingBufferedWriter extends Writer implements CharSequence {
 
@@ -203,41 +204,9 @@ final class TruncatingBufferedWriter extends Writer implements CharSequence {
 
     }
 
-    int indexOf(final CharSequence seq) {
-
-        // Short-circuit if there is nothing to match.
-        final int seqLength = seq.length();
-        if (seqLength == 0) {
-            return 0;
-        }
-
-        // Short-circuit if the given input is longer than the buffer.
-        if (seqLength > position) {
-            return -1;
-        }
-
-        // Perform the search.
-        for (int bufferIndex = 0; bufferIndex < position; bufferIndex++) {
-            boolean found = true;
-            for (int seqIndex = 0; seqIndex < seqLength; seqIndex++) {
-                final char s = seq.charAt(seqIndex);
-                final char b = buffer[bufferIndex + seqIndex];
-                if (s != b) {
-                    found = false;
-                    break;
-                }
-            }
-            if (found) {
-                return bufferIndex;
-            }
-        }
-        return -1;
-
-    }
-
     @Override
     public int length() {
-        return position + 1;
+        return position;
     }
 
     @Override
@@ -247,7 +216,20 @@ final class TruncatingBufferedWriter extends Writer implements CharSequence {
 
     @Override
     public String subSequence(final int startIndex, final int endIndex) {
-        return new String(buffer, startIndex, endIndex - startIndex);
+        throw new UnsupportedOperationException(
+                "operation requires allocation, contradicting with the purpose of the class");
+    }
+
+    @Override
+    public IntStream chars() {
+        throw new UnsupportedOperationException(
+                "operation requires allocation, contradicting with the purpose of the class");
+    }
+
+    @Override
+    public IntStream codePoints() {
+        throw new UnsupportedOperationException(
+                "operation requires allocation, contradicting with the purpose of the class");
     }
 
     @Override
diff --git a/log4j-layout-template-json/src/test/java/org/apache/logging/log4j/layout/template/json/resolver/StackTraceStringResolverTest.java b/log4j-layout-template-json/src/test/java/org/apache/logging/log4j/layout/template/json/resolver/StackTraceStringResolverTest.java
new file mode 100644
index 0000000000..c1aa485e21
--- /dev/null
+++ b/log4j-layout-template-json/src/test/java/org/apache/logging/log4j/layout/template/json/resolver/StackTraceStringResolverTest.java
@@ -0,0 +1,437 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF 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.apache.logging.log4j.layout.template.json.resolver;
+
+import org.apache.logging.log4j.core.LogEvent;
+import org.apache.logging.log4j.core.impl.Log4jLogEvent;
+import org.apache.logging.log4j.layout.template.json.JsonTemplateLayout;
+import org.assertj.core.api.AbstractStringAssert;
+import org.assertj.core.api.Assertions;
+import org.junit.jupiter.api.Nested;
+import org.junit.jupiter.api.Test;
+
+import java.io.ByteArrayOutputStream;
+import java.io.PrintStream;
+import java.math.BigDecimal;
+import java.net.ServerSocket;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.apache.logging.log4j.layout.template.json.TestHelpers.*;
+import static org.assertj.core.api.Assertions.assertThat;
+
+class StackTraceStringResolverTest {
+
+    ////////////////////////////////////////////////////////////////////////////
+    // exceptions //////////////////////////////////////////////////////////////
+    ////////////////////////////////////////////////////////////////////////////
+
+    // Below we create arbitrary exceptions containing stack entries from non-Log4j packages.
+    // Non-Log4j package origin is needed to avoid the truncation (e.g., `... 58 more`) done by `Throwable#printStackTrace()`.
+
+    private static final String EXCEPTION_REGEX_FLAGS = "(?m)(?s)";     // MULTILINE | DOTALL
+
+    private static final String TRUNCATION_SUFFIX = "<truncated>";
+
+    @SuppressWarnings({"BigDecimalMethodWithoutRoundingCalled", "ResultOfMethodCallIgnored"})
+    private static Throwable exception1() {
+        return catchException(() -> BigDecimal.ONE.divide(BigDecimal.ZERO));
+    }
+
+    private static String exception1Regex(final boolean truncated) {
+        final String truncationCorrectionRegex = truncationSuffixRegexOr(truncated, ".divide\\(");
+        return "java.lang.ArithmeticException: Division by zero\r?\n" +
+                "\t+at java.math.BigDecimal" + truncationCorrectionRegex + ".*";
+    }
+
+    @SuppressWarnings("ConstantConditions")
+    private static Throwable exception2() {
+        return catchException(() -> Collections.emptyList().add(0));
+    }
+
+    private static String exception2Regex(final boolean truncated) {
+        final String truncationCorrectionRegex = truncationSuffixRegexOr(truncated, ".add\\(");
+        return "java.lang.UnsupportedOperationException\r?\n" +
+                "\t+at java.util.AbstractList" + truncationCorrectionRegex + ".*";
+    }
+
+    private static Throwable exception3() {
+        return catchException(() -> new ServerSocket(-1));
+    }
+
+    private static String exception3Regex(final boolean truncated) {
+        final String truncationCorrectionRegex = truncationSuffixRegexOr(truncated, ".<init>");
+        return "java.lang.IllegalArgumentException: Port value out of range: -1\r?\n" +
+                "\t+at java.net.ServerSocket" + truncationCorrectionRegex + ".*";
+    }
+
+    private static String truncationSuffixRegexOr(final boolean truncated, final String fallback) {
+        return truncated
+                ? ("\r?\n" + TRUNCATION_SUFFIX)
+                : fallback;
+    }
+
+    private static Throwable catchException(ThrowingRunnable runnable) {
+        try {
+            runnable.run();
+            throw new AssertionError("should not have reached here");
+        } catch (Throwable error) {
+            return error;
+        }
+    }
+
+    @FunctionalInterface
+    private interface ThrowingRunnable {
+
+        void run() throws Throwable;
+
+    }
+
+    @Test
+    void exception1_regex_should_match() {
+        final Throwable error = exception1();
+        final String stackTrace = stackTrace(error);
+        final String regex = exception1Regex(false);
+        Assertions
+                .assertThat(stackTrace)
+                .matches(EXCEPTION_REGEX_FLAGS + regex);
+    }
+
+    @Test
+    void exception2_regex_should_match() {
+        final Throwable error = exception2();
+        final String stackTrace = stackTrace(error);
+        final String regex = exception2Regex(false);
+        Assertions
+                .assertThat(stackTrace)
+                .matches(EXCEPTION_REGEX_FLAGS + regex);
+    }
+
+    @Test
+    void exception3_regex_should_match() {
+        final Throwable error = exception3();
+        final String stackTrace = stackTrace(error);
+        final String regex = exception3Regex(false);
+        Assertions
+                .assertThat(stackTrace)
+                .matches(EXCEPTION_REGEX_FLAGS + regex);
+    }
+
+    private static String stackTrace(final Throwable throwable) {
+        final String encoding = "UTF-8";
+        try (ByteArrayOutputStream outputStream = new ByteArrayOutputStream();
+             PrintStream printStream = new PrintStream(outputStream, false, encoding)) {
+            throwable.printStackTrace(printStream);
+            printStream.flush();
+            return outputStream.toString(encoding);
+        } catch (Exception error) {
+            throw new RuntimeException(error);
+        }
+    }
+
+    ////////////////////////////////////////////////////////////////////////////
+    // abstract tests //////////////////////////////////////////////////////////
+    ////////////////////////////////////////////////////////////////////////////
+
+    private static abstract class AbstractTestCases {
+
+        private final boolean truncated;
+
+        AbstractTestCases(boolean truncated) {
+            this.truncated = truncated;
+        }
+
+        private String exception1Regex() {
+            return StackTraceStringResolverTest.exception1Regex(truncated);
+        }
+
+        private String exception2Regex() {
+            return StackTraceStringResolverTest.exception2Regex(truncated);
+        }
+
+        private String exception3Regex() {
+            return StackTraceStringResolverTest.exception3Regex(truncated);
+        }
+
+        @Test
+        void exception_should_be_resolved() {
+            final Throwable exception = exception1();
+            final String serializedExceptionRegex = EXCEPTION_REGEX_FLAGS + exception1Regex();
+            assertSerializedException(exception, serializedExceptionRegex);
+        }
+
+        @Test
+        void exception_with_cause_should_be_resolved() {
+
+            // Create the exception.
+            final Throwable exception = exception1();
+            final Throwable cause = exception2();
+            exception.initCause(cause);
+
+            // Check the serialized exception.
+            final String serializedExceptionRegex = EXCEPTION_REGEX_FLAGS +
+                    exception1Regex() +
+                    "\nCaused by: " + exception2Regex();
+            assertSerializedException(exception, serializedExceptionRegex);
+
+        }
+
+        @Test
+        void exception_with_causes_should_be_resolved() {
+
+            // Create the exception.
+            final Throwable exception = exception1();
+            final Throwable cause1 = exception2();
+            final Throwable cause2 = exception3();
+            exception.initCause(cause1);
+            cause1.initCause(cause2);
+
+            // Check the serialized exception.
+            final String serializedExceptionRegex = EXCEPTION_REGEX_FLAGS +
+                    exception1Regex() +
+                    "\nCaused by: " + exception2Regex() +
+                    "\nCaused by: " + exception3Regex();
+            assertSerializedException(exception, serializedExceptionRegex);
+
+        }
+
+        @Test
+        void exception_with_suppressed_should_be_resolved() {
+
+            // Create the exception.
+            final Throwable exception = exception1();
+            final Throwable suppressed = exception2();
+            exception.addSuppressed(suppressed);
+
+            // Check the serialized exception.
+            final String serializedExceptionRegex = EXCEPTION_REGEX_FLAGS +
+                    exception1Regex() +
+                    "\n\tSuppressed: " + exception2Regex();
+            assertSerializedException(exception, serializedExceptionRegex);
+
+        }
+
+        @Test
+        void exception_with_suppresseds_should_be_resolved() {
+
+            // Create the exception.
+            final Throwable exception = exception1();
+            final Throwable suppressed1 = exception2();
+            final Throwable suppressed2 = exception3();
+            exception.addSuppressed(suppressed1);
+            exception.addSuppressed(suppressed2);
+
+            // Check the serialized exception.
+            final String serializedExceptionRegex = EXCEPTION_REGEX_FLAGS +
+                    exception1Regex() +
+                    "\n\tSuppressed: " + exception2Regex() +
+                    "\n\tSuppressed: " + exception3Regex();
+            assertSerializedException(exception, serializedExceptionRegex);
+
+        }
+
+        @Test
+        void exception_with_cause_and_suppressed_should_be_resolved() {
+
+            // Create the exception.
+            final Throwable exception = exception1();
+            final Throwable suppressed = exception2();
+            final Throwable cause = exception3();
+            exception.addSuppressed(suppressed);
+            exception.initCause(cause);
+
+            // Check the serialized exception.
+            final String serializedExceptionRegex = EXCEPTION_REGEX_FLAGS +
+                    exception1Regex() +
+                    "\n\tSuppressed: " + exception2Regex() +
+                    "\nCaused by: " + exception3Regex();
+            assertSerializedException(exception, serializedExceptionRegex);
+
+        }
+
+        @Test
+        void exception_with_cause_with_suppressed_should_be_resolved() {
+
+            // Create the exception.
+            final Throwable exception = exception1();
+            final Throwable cause = exception2();
+            final Throwable suppressed = exception3();
+            exception.initCause(cause);
+            cause.addSuppressed(suppressed);
+
+            // Check the serialized exception.
+            final String serializedExceptionRegex = EXCEPTION_REGEX_FLAGS +
+                    exception1Regex() +
+                    "\nCaused by: " + exception2Regex() +
+                    "\n\tSuppressed: " + exception3Regex();
+            assertSerializedException(exception, serializedExceptionRegex);
+
+        }
+
+        @Test
+        void exception_with_suppressed_with_cause_should_be_resolved() {
+
+            // Create the exception.
+            final Throwable exception = exception1();
+            final Throwable suppressed = exception2();
+            final Throwable cause = exception3();
+            exception.addSuppressed(suppressed);
+            suppressed.initCause(cause);
+
+            // Check the serialized exception.
+            final String serializedExceptionRegex = EXCEPTION_REGEX_FLAGS +
+                    exception1Regex() +
+                    "\n\tSuppressed: " + exception2Regex() +
+                    "\n\tCaused by: " + exception3Regex();
+            assertSerializedException(exception, serializedExceptionRegex);
+
+        }
+
+        abstract void assertSerializedException(
+                final Throwable exception,
+                final String regex);
+
+    }
+
+    ////////////////////////////////////////////////////////////////////////////
+    // tests without truncation ////////////////////////////////////////////////
+    ////////////////////////////////////////////////////////////////////////////
+
+    @Nested
+    class WithoutTruncation extends AbstractTestCases {
+
+        WithoutTruncation() {
+            super(false);
+        }
+
+        @Override
+        void assertSerializedException(final Throwable exception, final String regex) {
+            assertSerializedExceptionWithoutTruncation(exception, regex);
+        }
+
+    }
+
+    private static void assertSerializedExceptionWithoutTruncation(
+            final Throwable exception,
+            final String regex) {
+
+        // Create the event template.
+        final Map<String, ?> exceptionResolverTemplate = asMap(
+                "$resolver", "exception",
+                "field", "stackTrace",
+                "stackTrace", asMap("stringified", true));
+
+        // Check the serialized event.
+        assertSerializedException(
+                exceptionResolverTemplate,
+                exception,
+                serializedExceptionAssert -> serializedExceptionAssert.matches(regex));
+
+    }
+
+    ////////////////////////////////////////////////////////////////////////////
+    // tests with `truncationPointMatcherStrings` //////////////////////////////
+    ////////////////////////////////////////////////////////////////////////////
+
+    @Nested
+    class WithStringTruncation extends AbstractTestCases {
+
+        WithStringTruncation() {
+            super(true);
+        }
+
+        @Override
+        void assertSerializedException(final Throwable exception, final String regex) {
+            assertSerializedExceptionWithStringTruncation(exception, regex);
+        }
+        
+    }
+
+    private static void assertSerializedExceptionWithStringTruncation(
+            final Throwable exception,
+            final String regex) {
+
+        // Create the event template.
+        final List<String> pointMatcherStrings = pointMatcherStrings();
+        final Map<String, ?> exceptionResolverTemplate = asMap(
+                "$resolver", "exception",
+                "field", "stackTrace",
+                "stackTrace", asMap("stringified", asMap(
+                        "truncation", asMap(
+                                "suffix", TRUNCATION_SUFFIX,
+                                "pointMatcherStrings", pointMatcherStrings))));
+
+        // Check the serialized event.
+        assertSerializedException(
+                exceptionResolverTemplate,
+                exception,
+                serializedExceptionAssert -> serializedExceptionAssert.matches(regex));
+
+    }
+
+    private static List<String> pointMatcherStrings() {
+        final Throwable exception1 = exception1();
+        final Throwable exception2 = exception2();
+        final Throwable exception3 = exception3();
+        return Stream
+                .of(exception1, exception2, exception3)
+                .map(exception -> {
+                    final StackTraceElement stackTraceElement = exception.getStackTrace()[0];
+                    final String className = stackTraceElement.getClassName();
+                    return "at " + className;
+                })
+                .collect(Collectors.toList());
+    }
+
+    ////////////////////////////////////////////////////////////////////////////
+    // utilities ///////////////////////////////////////////////////////////////
+    ////////////////////////////////////////////////////////////////////////////
+
+    private static void assertSerializedException(
+            final Map<String, ?> exceptionResolverTemplate,
+            final Throwable exception,
+            final Consumer<AbstractStringAssert<?>> serializedExceptionAsserter) {
+
+        // Create the event template.
+        final String eventTemplate = writeJson(asMap("output", exceptionResolverTemplate));
+
+        // Create the layout.
+        final JsonTemplateLayout layout = JsonTemplateLayout
+                .newBuilder()
+                .setConfiguration(CONFIGURATION)
+                .setEventTemplate(eventTemplate)
+                .build();
+
+        // Create the log event.
+        final LogEvent logEvent = Log4jLogEvent
+                .newBuilder()
+                .setThrown(exception)
+                .build();
+
+        // Check the serialized event.
+        usingSerializedLogEventAccessor(layout, logEvent, accessor -> {
+            AbstractStringAssert<?> serializedExceptionAssert = assertThat(accessor.getString("output"));
+            serializedExceptionAsserter.accept(serializedExceptionAssert);
+        });
+
+    }
+
+}
diff --git a/log4j-layout-template-json/src/test/java/org/apache/logging/log4j/layout/template/json/util/CharSequencePointerTest.java b/log4j-layout-template-json/src/test/java/org/apache/logging/log4j/layout/template/json/util/CharSequencePointerTest.java
new file mode 100644
index 0000000000..b00270a525
--- /dev/null
+++ b/log4j-layout-template-json/src/test/java/org/apache/logging/log4j/layout/template/json/util/CharSequencePointerTest.java
@@ -0,0 +1,121 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF 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.apache.logging.log4j.layout.template.json.util;
+
+import org.assertj.core.api.Assertions;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.CsvSource;
+
+class CharSequencePointerTest {
+
+    private final CharSequencePointer pointer = new CharSequencePointer();
+
+    @Test
+    void length_should_fail_without_reset() {
+        // noinspection ResultOfMethodCallIgnored
+        assertMissingReset(pointer::length);
+    }
+
+    @Test
+    void charAt_should_fail_without_reset() {
+        assertMissingReset(() -> pointer.charAt(0));
+    }
+
+    @Test
+    void toString_should_fail_without_reset() {
+        // noinspection ResultOfMethodCallIgnored
+        assertMissingReset(pointer::toString);
+    }
+
+    private static void assertMissingReset(final Runnable runnable) {
+        Assertions
+                .assertThatThrownBy(runnable::run)
+                .isInstanceOf(IllegalStateException.class)
+                .hasMessage("pointer must be reset first");
+    }
+
+    @ParameterizedTest
+    @CsvSource({
+            "'',0,0,''",
+            "foo,0,1,f",
+            "foo,1,1,''",
+            "foo,1,2,o",
+            "foo,3,3,''"
+    })
+    void toString_should_subSequence(
+            final CharSequence delegate,
+            final int startIndex,
+            final int endIndex,
+            final String expectedOutput) {
+        pointer.reset(delegate, startIndex, endIndex);
+        Assertions.assertThat(pointer).hasToString(expectedOutput);
+    }
+
+    @Test
+    void subSequence_should_not_be_supported() {
+        pointer.reset("", 0, 0);
+        assertUnsupportedOperation(() -> pointer.subSequence(0, 0));
+    }
+
+    @Test
+    void chars_should_not_be_supported() {
+        pointer.reset("", 0, 0);
+        assertUnsupportedOperation(() -> pointer.subSequence(0, 0));
+    }
+
+    @Test
+    void codePoints_should_not_be_supported() {
+        pointer.reset("", 0, 0);
+        assertUnsupportedOperation(() -> pointer.subSequence(0, 0));
+    }
+
+    private static void assertUnsupportedOperation(final Runnable runnable) {
+        Assertions
+                .assertThatThrownBy(runnable::run)
+                .isInstanceOf(UnsupportedOperationException.class)
+                .hasMessage("operation requires allocation, contradicting with the purpose of the class");
+    }
+
+    @Test
+    void reset_should_fail_on_null_delegate() {
+        Assertions
+                .assertThatThrownBy(() -> pointer.reset(null, 0, 0))
+                .isInstanceOf(NullPointerException.class)
+                .hasMessage("delegate");
+    }
+
+    @ParameterizedTest
+    @CsvSource({
+            "foo,-1,3,invalid start: -1",
+            "foo,4,3,invalid length: -1",
+            "foo,0,-1,invalid length: -1",
+            "foo,1,0,invalid length: -1",
+            "foo,0,4,invalid end: 4"
+    })
+    void reset_should_fail_on_invalid_indices(
+            final CharSequence delegate,
+            final int startIndex,
+            final int endIndex,
+            final String expectedErrorMessage) {
+        Assertions
+                .assertThatThrownBy(() -> pointer.reset(delegate, startIndex, endIndex))
+                .isInstanceOf(IndexOutOfBoundsException.class)
+                .hasMessage(expectedErrorMessage);
+    }
+
+}
diff --git a/log4j-layout-template-json/src/test/java/org/apache/logging/log4j/layout/template/json/util/TruncatingBufferedWriterTest.java b/log4j-layout-template-json/src/test/java/org/apache/logging/log4j/layout/template/json/util/TruncatingBufferedWriterTest.java
index b52d453a57..113e8dd5af 100644
--- a/log4j-layout-template-json/src/test/java/org/apache/logging/log4j/layout/template/json/util/TruncatingBufferedWriterTest.java
+++ b/log4j-layout-template-json/src/test/java/org/apache/logging/log4j/layout/template/json/util/TruncatingBufferedWriterTest.java
@@ -19,6 +19,8 @@ package org.apache.logging.log4j.layout.template.json.util;
 import org.assertj.core.api.Assertions;
 import org.junit.jupiter.api.Test;
 
+import java.util.function.Consumer;
+
 class TruncatingBufferedWriterTest {
 
     @Test
@@ -225,7 +227,7 @@ class TruncatingBufferedWriterTest {
         verifyTruncation(writer, 'n');
     }
 
-    private void verifyTruncation(
+    private static void verifyTruncation(
             final TruncatingBufferedWriter writer,
             final char c) {
         Assertions.assertThat(writer.buffer()).isEqualTo(new char[]{c});
@@ -235,10 +237,67 @@ class TruncatingBufferedWriterTest {
         verifyClose(writer);
     }
 
-    private void verifyClose(final TruncatingBufferedWriter writer) {
+    private static void verifyClose(final TruncatingBufferedWriter writer) {
         writer.close();
         Assertions.assertThat(writer.position()).isEqualTo(0);
         Assertions.assertThat(writer.truncated()).isFalse();
     }
 
+    @Test
+    void test_length_and_position() {
+
+        // Create the writer and the verifier.
+        final TruncatingBufferedWriter writer = new TruncatingBufferedWriter(2);
+        final Consumer<Integer> positionAndLengthVerifier =
+                (final Integer expected) -> Assertions
+                        .assertThat(writer.position())
+                        .isEqualTo(writer.length())
+                        .isEqualTo(expected);
+
+        // Check the initial condition.
+        positionAndLengthVerifier.accept(0);
+
+        // Append the 1st character and verify.
+        writer.write("a");
+        positionAndLengthVerifier.accept(1);
+
+        // Append the 2nd character and verify.
+        writer.write("b");
+        positionAndLengthVerifier.accept(2);
+
+        // Append the 3rd to-be-truncated character and verify.
+        writer.write("c");
+        positionAndLengthVerifier.accept(2);
+
+        // Reposition the writer and verify.
+        writer.position(1);
+        positionAndLengthVerifier.accept(1);
+
+    }
+
+    @Test
+    void subSequence_should_not_be_supported() {
+        final TruncatingBufferedWriter writer = new TruncatingBufferedWriter(2);
+        assertUnsupportedOperation(() -> writer.subSequence(0, 0));
+    }
+
+    @Test
+    void chars_should_not_be_supported() {
+        final TruncatingBufferedWriter writer = new TruncatingBufferedWriter(2);
+        assertUnsupportedOperation(() -> writer.subSequence(0, 0));
+    }
+
+    @Test
+    void codePoints_should_not_be_supported() {
+        final TruncatingBufferedWriter writer = new TruncatingBufferedWriter(2);
+        assertUnsupportedOperation(() -> writer.subSequence(0, 0));
+    }
+
+    private static void assertUnsupportedOperation(final Runnable runnable) {
+        Assertions
+                .assertThatThrownBy(runnable::run)
+                .isInstanceOf(UnsupportedOperationException.class)
+                .hasMessage("operation requires allocation, contradicting with the purpose of the class");
+    }
+
 }
diff --git a/src/changes/changes.xml b/src/changes/changes.xml
index de42842d74..056235d53a 100644
--- a/src/changes/changes.xml
+++ b/src/changes/changes.xml
@@ -30,6 +30,9 @@
          - "remove" - Removed
     -->
     <release version="2.18.1" date="2022-MM-DD" description="GA Release 2.18.1">
+      <action issue="LOG4J2-3556" dev="vy" type="fix" due-to=" Arthur Gavlyukovskiy">
+        Make JsonTemplateLayout stack trace truncation operate for each label block.
+      </action>
       <action issue="LOG4J2-3550" dev="rgoers" type="fix" due-to="DongjianPeng">
         SystemPropertyAribiter was assigning the value as the name.
       </action>
diff --git a/src/site/asciidoc/manual/json-template-layout.adoc.vm b/src/site/asciidoc/manual/json-template-layout.adoc.vm
index 53410b2c6f..2fbce39a54 100644
--- a/src/site/asciidoc/manual/json-template-layout.adoc.vm
+++ b/src/site/asciidoc/manual/json-template-layout.adoc.vm
@@ -702,9 +702,13 @@ are provided.
 stringified stack traces after the given matching point. If both parameters are
 provided, `pointMatcherStrings` will be checked first.
 
-If a stringified stack trace truncation takes place, it will be indicated with
+If a stringified stack trace truncation takes place, it will be indicated with a
 `suffix`, which by default is set to the configured `truncatedStringSuffix` in
-the layout, unless explicitly provided.
+the layout, unless explicitly provided. Every truncation suffix is prefixed with
+a newline.
+
+Stringified stack trace truncation operates in `Caused by:` and `Suppressed:`
+label blocks. That is, matchers are executed against each label in isolation.
 
 `elementTemplate` is an object describing the template to be used while
 resolving the `StackTraceElement` array. If `stringified` is set to `true`,
@@ -783,7 +787,7 @@ truncated after the given point matcher:
   "stackTrace": {
     "stringified": {
       "truncation": {
-        "suffix": ">",
+        "suffix": "... [truncated]",
         "pointMatcherStrings": ["at javax.servlet.http.HttpServlet.service"]
       }
     }