From b458fe35733092b32599ca2fbc31757d7d78be5c Mon Sep 17 00:00:00 2001 From: Ming-Yen Chung Date: Thu, 23 Oct 2025 00:31:39 +0800 Subject: [PATCH 01/10] Make RecordHeader Read Thread-Safe --- .../common/header/internals/RecordHeader.java | 22 +++++++--- .../header/internals/RecordHeadersTest.java | 43 +++++++++++++++++++ 2 files changed, 58 insertions(+), 7 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/header/internals/RecordHeader.java b/clients/src/main/java/org/apache/kafka/common/header/internals/RecordHeader.java index e1ce6ad01a5bb..3eee4d27da50a 100644 --- a/clients/src/main/java/org/apache/kafka/common/header/internals/RecordHeader.java +++ b/clients/src/main/java/org/apache/kafka/common/header/internals/RecordHeader.java @@ -25,9 +25,9 @@ public class RecordHeader implements Header { private ByteBuffer keyBuffer; - private String key; - private ByteBuffer valueBuffer; - private byte[] value; + private volatile String key; + private volatile ByteBuffer valueBuffer; + private volatile byte[] value; public RecordHeader(String key, byte[] value) { Objects.requireNonNull(key, "Null header keys are not permitted"); @@ -42,16 +42,24 @@ public RecordHeader(ByteBuffer keyBuffer, ByteBuffer valueBuffer) { public String key() { if (key == null) { - key = Utils.utf8(keyBuffer, keyBuffer.remaining()); - keyBuffer = null; + synchronized (this) { + if (key == null) { + key = Utils.utf8(keyBuffer, keyBuffer.remaining()); + keyBuffer = null; + } + } } return key; } public byte[] value() { if (value == null && valueBuffer != null) { - value = Utils.toArray(valueBuffer); - valueBuffer = null; + synchronized (this) { + if (value == null && valueBuffer != null) { + value = Utils.toArray(valueBuffer); + valueBuffer = null; + } + } } return value; } diff --git a/clients/src/test/java/org/apache/kafka/common/header/internals/RecordHeadersTest.java b/clients/src/test/java/org/apache/kafka/common/header/internals/RecordHeadersTest.java index 41104194991d9..45f1ce3b5c573 100644 --- a/clients/src/test/java/org/apache/kafka/common/header/internals/RecordHeadersTest.java +++ b/clients/src/test/java/org/apache/kafka/common/header/internals/RecordHeadersTest.java @@ -22,7 +22,14 @@ import org.junit.jupiter.api.Test; import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; import java.util.Iterator; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; import static org.junit.jupiter.api.Assertions.assertArrayEquals; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -265,4 +272,40 @@ static void assertHeader(String key, String value, Header actual) { assertArrayEquals(value.getBytes(), actual.value()); } + @Test + public void testRecordHeaderIsReadThreadSafe() throws Exception { + int repeats = 5000; + int threads = 8; + + for (int test = 0; test < repeats; test++) { + RecordHeader header = new RecordHeader( + ByteBuffer.wrap("key".getBytes(StandardCharsets.UTF_8)), + ByteBuffer.wrap("value".getBytes(StandardCharsets.UTF_8)) + ); + + ExecutorService pool = Executors.newFixedThreadPool(threads); + CountDownLatch startLatch = new CountDownLatch(1); + AtomicBoolean raceDetected = new AtomicBoolean(false); + + Runnable task = () -> { + try { + startLatch.await(); + header.key(); + header.value(); + } catch (NullPointerException e) { + raceDetected.set(true); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + }; + + for (int i = 0; i < threads; i++) pool.submit(task); + + startLatch.countDown(); + pool.shutdown(); + pool.awaitTermination(5, TimeUnit.SECONDS); + + assertFalse(raceDetected.get(), "Read race condition detected in RecordHeader!"); + } + } } From d8974925793e88d581d538f7e322a05d1c3578b0 Mon Sep 17 00:00:00 2001 From: Ming-Yen Chung Date: Thu, 23 Oct 2025 01:14:08 +0800 Subject: [PATCH 02/10] Use RepeatedTest instead --- .../header/internals/RecordHeadersTest.java | 63 +++++++++---------- 1 file changed, 30 insertions(+), 33 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/common/header/internals/RecordHeadersTest.java b/clients/src/test/java/org/apache/kafka/common/header/internals/RecordHeadersTest.java index 45f1ce3b5c573..acf47b1e090c4 100644 --- a/clients/src/test/java/org/apache/kafka/common/header/internals/RecordHeadersTest.java +++ b/clients/src/test/java/org/apache/kafka/common/header/internals/RecordHeadersTest.java @@ -19,6 +19,7 @@ import org.apache.kafka.common.header.Header; import org.apache.kafka.common.header.Headers; +import org.junit.jupiter.api.RepeatedTest; import org.junit.jupiter.api.Test; import java.io.IOException; @@ -272,40 +273,36 @@ static void assertHeader(String key, String value, Header actual) { assertArrayEquals(value.getBytes(), actual.value()); } - @Test + @RepeatedTest(100) public void testRecordHeaderIsReadThreadSafe() throws Exception { - int repeats = 5000; int threads = 8; - - for (int test = 0; test < repeats; test++) { - RecordHeader header = new RecordHeader( - ByteBuffer.wrap("key".getBytes(StandardCharsets.UTF_8)), - ByteBuffer.wrap("value".getBytes(StandardCharsets.UTF_8)) - ); - - ExecutorService pool = Executors.newFixedThreadPool(threads); - CountDownLatch startLatch = new CountDownLatch(1); - AtomicBoolean raceDetected = new AtomicBoolean(false); - - Runnable task = () -> { - try { - startLatch.await(); - header.key(); - header.value(); - } catch (NullPointerException e) { - raceDetected.set(true); - } catch (InterruptedException e) { - throw new RuntimeException(e); - } - }; - - for (int i = 0; i < threads; i++) pool.submit(task); - - startLatch.countDown(); - pool.shutdown(); - pool.awaitTermination(5, TimeUnit.SECONDS); - - assertFalse(raceDetected.get(), "Read race condition detected in RecordHeader!"); - } + RecordHeader header = new RecordHeader( + ByteBuffer.wrap("key".getBytes(StandardCharsets.UTF_8)), + ByteBuffer.wrap("value".getBytes(StandardCharsets.UTF_8)) + ); + + ExecutorService pool = Executors.newFixedThreadPool(threads); + CountDownLatch startLatch = new CountDownLatch(1); + AtomicBoolean raceDetected = new AtomicBoolean(false); + + Runnable task = () -> { + try { + startLatch.await(); + header.key(); + header.value(); + } catch (NullPointerException e) { + raceDetected.set(true); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + }; + + for (int i = 0; i < threads; i++) pool.submit(task); + + startLatch.countDown(); + pool.shutdown(); + pool.awaitTermination(5, TimeUnit.SECONDS); + + assertFalse(raceDetected.get(), "Read race condition detected in RecordHeader!"); } } From aeb369e8271a59e579becee1d2c70a84b8b1bd39 Mon Sep 17 00:00:00 2001 From: Ming-Yen Chung Date: Thu, 23 Oct 2025 11:46:51 +0800 Subject: [PATCH 03/10] Update upgrade.html --- docs/upgrade.html | 3 +++ 1 file changed, 3 insertions(+) diff --git a/docs/upgrade.html b/docs/upgrade.html index 80343474d7051..2a00a0cdd6cfd 100644 --- a/docs/upgrade.html +++ b/docs/upgrade.html @@ -25,6 +25,9 @@
Upgrading Servers to 4
Notable changes in 4.2.0
    +
  • + The org.apache.kafka.common.header.internals.RecordHeader class has been updated to be read thread-safe. See KIP-1205 for details. +
  • The org.apache.kafka.disallowed.login.modules config was deprecated. Please use the org.apache.kafka.allowed.login.modules instead. From 3b72021233e811e30dbd55de92f148442f852a6c Mon Sep 17 00:00:00 2001 From: Ming-Yen Chung Date: Sat, 25 Oct 2025 00:51:32 +0800 Subject: [PATCH 04/10] Use try-finally to ensure resource release --- .../header/internals/RecordHeadersTest.java | 44 +++++++++++-------- 1 file changed, 26 insertions(+), 18 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/common/header/internals/RecordHeadersTest.java b/clients/src/test/java/org/apache/kafka/common/header/internals/RecordHeadersTest.java index acf47b1e090c4..e7b17091caaa8 100644 --- a/clients/src/test/java/org/apache/kafka/common/header/internals/RecordHeadersTest.java +++ b/clients/src/test/java/org/apache/kafka/common/header/internals/RecordHeadersTest.java @@ -285,24 +285,32 @@ public void testRecordHeaderIsReadThreadSafe() throws Exception { CountDownLatch startLatch = new CountDownLatch(1); AtomicBoolean raceDetected = new AtomicBoolean(false); - Runnable task = () -> { - try { - startLatch.await(); - header.key(); - header.value(); - } catch (NullPointerException e) { - raceDetected.set(true); - } catch (InterruptedException e) { - throw new RuntimeException(e); + try { + Runnable task = () -> { + try { + startLatch.await(); + header.key(); + header.value(); + } catch (NullPointerException e) { + raceDetected.set(true); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException(e); + } + }; + + for (int i = 0; i < threads; i++) pool.submit(task); + + startLatch.countDown(); + + pool.shutdown(); + pool.awaitTermination(5, TimeUnit.SECONDS); + + assertFalse(raceDetected.get(), "Read race condition detected in RecordHeader!"); + } finally { + if (!pool.isTerminated()) { + pool.shutdownNow(); } - }; - - for (int i = 0; i < threads; i++) pool.submit(task); - - startLatch.countDown(); - pool.shutdown(); - pool.awaitTermination(5, TimeUnit.SECONDS); - - assertFalse(raceDetected.get(), "Read race condition detected in RecordHeader!"); + } } } From 07b801459bde4d03e4e1ee264bd6114c161d7f98 Mon Sep 17 00:00:00 2001 From: Ming-Yen Chung Date: Wed, 29 Oct 2025 17:30:01 +0800 Subject: [PATCH 05/10] Use CompletableFuture and add test for null value --- .../header/internals/RecordHeadersTest.java | 57 +++++++++++++------ 1 file changed, 39 insertions(+), 18 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/common/header/internals/RecordHeadersTest.java b/clients/src/test/java/org/apache/kafka/common/header/internals/RecordHeadersTest.java index e7b17091caaa8..f38c8c8f53058 100644 --- a/clients/src/test/java/org/apache/kafka/common/header/internals/RecordHeadersTest.java +++ b/clients/src/test/java/org/apache/kafka/common/header/internals/RecordHeadersTest.java @@ -26,11 +26,11 @@ import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; import java.util.Iterator; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.stream.Collectors; +import java.util.stream.IntStream; import static org.junit.jupiter.api.Assertions.assertArrayEquals; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -275,18 +275,17 @@ static void assertHeader(String key, String value, Header actual) { @RepeatedTest(100) public void testRecordHeaderIsReadThreadSafe() throws Exception { - int threads = 8; + int threads = 32; RecordHeader header = new RecordHeader( ByteBuffer.wrap("key".getBytes(StandardCharsets.UTF_8)), ByteBuffer.wrap("value".getBytes(StandardCharsets.UTF_8)) ); - ExecutorService pool = Executors.newFixedThreadPool(threads); CountDownLatch startLatch = new CountDownLatch(1); AtomicBoolean raceDetected = new AtomicBoolean(false); - try { - Runnable task = () -> { + var fs = IntStream.range(0, threads) + .mapToObj(i -> CompletableFuture.runAsync(() -> { try { startLatch.await(); header.key(); @@ -297,20 +296,42 @@ public void testRecordHeaderIsReadThreadSafe() throws Exception { Thread.currentThread().interrupt(); throw new RuntimeException(e); } - }; + })).collect(Collectors.toUnmodifiableList()); - for (int i = 0; i < threads; i++) pool.submit(task); + startLatch.countDown(); + fs.forEach(CompletableFuture::join); - startLatch.countDown(); + assertFalse(raceDetected.get(), "Read race condition detected in RecordHeader!"); + } + + @RepeatedTest(100) + public void testRecordHeaderWithNullValueIsReadThreadSafe() throws Exception { + int threadCount = 32; + RecordHeader header = new RecordHeader( + ByteBuffer.wrap("key".getBytes(StandardCharsets.UTF_8)), + null + ); + + CountDownLatch startLatch = new CountDownLatch(1); + AtomicBoolean raceDetected = new AtomicBoolean(false); + + var fs = IntStream.range(0, threadCount) + .mapToObj(i -> CompletableFuture.runAsync(() -> { + try { + startLatch.await(); + header.key(); + header.value(); // may be null, should not throw + } catch (NullPointerException e) { + raceDetected.set(true); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException(e); + } + })).collect(Collectors.toUnmodifiableList()); - pool.shutdown(); - pool.awaitTermination(5, TimeUnit.SECONDS); + startLatch.countDown(); + fs.forEach(CompletableFuture::join); - assertFalse(raceDetected.get(), "Read race condition detected in RecordHeader!"); - } finally { - if (!pool.isTerminated()) { - pool.shutdownNow(); - } - } + assertFalse(raceDetected.get(), "Read race condition detected when value is null in RecordHeader!"); } } From 9812dc2028355ff583f2f0797fd8d5ae81e3ace0 Mon Sep 17 00:00:00 2001 From: Ming-Yen Chung Date: Wed, 29 Oct 2025 22:52:12 +0800 Subject: [PATCH 06/10] Refactor test code and modify ConsumerRecord doc --- .../clients/consumer/ConsumerRecord.java | 5 +- .../header/internals/RecordHeadersTest.java | 49 ++++++------------- 2 files changed, 17 insertions(+), 37 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecord.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecord.java index 11360b0dac0c9..a4d6c9cd94b45 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecord.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecord.java @@ -40,9 +40,8 @@ *
* *

- * In particular, the {@link #headers()} method returns a mutable collection of headers. If multiple - * threads access or modify these headers concurrently, it may lead to race conditions or inconsistent - * states. It is the responsibility of the user to ensure that multi-threaded access is properly synchronized. + * However, the {@link #headers()} collection and individual header instances are + * read thread-safe, allowing concurrent access for reading without additional synchronization. * *

* Refer to the {@link KafkaConsumer} documentation for more details on multi-threaded consumption and processing strategies. diff --git a/clients/src/test/java/org/apache/kafka/common/header/internals/RecordHeadersTest.java b/clients/src/test/java/org/apache/kafka/common/header/internals/RecordHeadersTest.java index f38c8c8f53058..2e4cd88a9c02f 100644 --- a/clients/src/test/java/org/apache/kafka/common/header/internals/RecordHeadersTest.java +++ b/clients/src/test/java/org/apache/kafka/common/header/internals/RecordHeadersTest.java @@ -273,18 +273,12 @@ static void assertHeader(String key, String value, Header actual) { assertArrayEquals(value.getBytes(), actual.value()); } - @RepeatedTest(100) - public void testRecordHeaderIsReadThreadSafe() throws Exception { - int threads = 32; - RecordHeader header = new RecordHeader( - ByteBuffer.wrap("key".getBytes(StandardCharsets.UTF_8)), - ByteBuffer.wrap("value".getBytes(StandardCharsets.UTF_8)) - ); - + private void assertRecordHeaderReadThreadSafe(RecordHeader header) throws Exception { + int threadCount = 32; CountDownLatch startLatch = new CountDownLatch(1); AtomicBoolean raceDetected = new AtomicBoolean(false); - var fs = IntStream.range(0, threads) + var futures = IntStream.range(0, threadCount) .mapToObj(i -> CompletableFuture.runAsync(() -> { try { startLatch.await(); @@ -299,39 +293,26 @@ public void testRecordHeaderIsReadThreadSafe() throws Exception { })).collect(Collectors.toUnmodifiableList()); startLatch.countDown(); - fs.forEach(CompletableFuture::join); + futures.forEach(CompletableFuture::join); + + assertFalse(raceDetected.get()); + } - assertFalse(raceDetected.get(), "Read race condition detected in RecordHeader!"); + @RepeatedTest(100) + public void testRecordHeaderIsReadThreadSafe() throws Exception { + RecordHeader header = new RecordHeader( + ByteBuffer.wrap("key".getBytes(StandardCharsets.UTF_8)), + ByteBuffer.wrap("value".getBytes(StandardCharsets.UTF_8)) + ); + assertRecordHeaderReadThreadSafe(header); } @RepeatedTest(100) public void testRecordHeaderWithNullValueIsReadThreadSafe() throws Exception { - int threadCount = 32; RecordHeader header = new RecordHeader( ByteBuffer.wrap("key".getBytes(StandardCharsets.UTF_8)), null ); - - CountDownLatch startLatch = new CountDownLatch(1); - AtomicBoolean raceDetected = new AtomicBoolean(false); - - var fs = IntStream.range(0, threadCount) - .mapToObj(i -> CompletableFuture.runAsync(() -> { - try { - startLatch.await(); - header.key(); - header.value(); // may be null, should not throw - } catch (NullPointerException e) { - raceDetected.set(true); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw new RuntimeException(e); - } - })).collect(Collectors.toUnmodifiableList()); - - startLatch.countDown(); - fs.forEach(CompletableFuture::join); - - assertFalse(raceDetected.get(), "Read race condition detected when value is null in RecordHeader!"); + assertRecordHeaderReadThreadSafe(header); } } From 50a585e10a6457f069ab96685ca7d42a764602a7 Mon Sep 17 00:00:00 2001 From: Ming-Yen Chung Date: Wed, 29 Oct 2025 22:56:48 +0800 Subject: [PATCH 07/10] Refine description in upgrade.html --- docs/upgrade.html | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/upgrade.html b/docs/upgrade.html index 2a00a0cdd6cfd..f3eb793f038f3 100644 --- a/docs/upgrade.html +++ b/docs/upgrade.html @@ -27,6 +27,7 @@

Notable changes in 4
  • The org.apache.kafka.common.header.internals.RecordHeader class has been updated to be read thread-safe. See KIP-1205 for details. + In other words, ConsumerRecord headers can now be safely read from multiple threads concurrently.
  • The org.apache.kafka.disallowed.login.modules config was deprecated. Please use the org.apache.kafka.allowed.login.modules From 30493d12462c4bab961929a785f8eb3483939d37 Mon Sep 17 00:00:00 2001 From: Ming-Yen Chung Date: Thu, 30 Oct 2025 22:46:22 +0800 Subject: [PATCH 08/10] Refactor test code --- .../kafka/common/header/internals/RecordHeadersTest.java | 9 ++------- 1 file changed, 2 insertions(+), 7 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/common/header/internals/RecordHeadersTest.java b/clients/src/test/java/org/apache/kafka/common/header/internals/RecordHeadersTest.java index 2e4cd88a9c02f..56e8b41f903ec 100644 --- a/clients/src/test/java/org/apache/kafka/common/header/internals/RecordHeadersTest.java +++ b/clients/src/test/java/org/apache/kafka/common/header/internals/RecordHeadersTest.java @@ -273,10 +273,9 @@ static void assertHeader(String key, String value, Header actual) { assertArrayEquals(value.getBytes(), actual.value()); } - private void assertRecordHeaderReadThreadSafe(RecordHeader header) throws Exception { - int threadCount = 32; + private void assertRecordHeaderReadThreadSafe(RecordHeader header) { + int threadCount = 16; CountDownLatch startLatch = new CountDownLatch(1); - AtomicBoolean raceDetected = new AtomicBoolean(false); var futures = IntStream.range(0, threadCount) .mapToObj(i -> CompletableFuture.runAsync(() -> { @@ -284,8 +283,6 @@ private void assertRecordHeaderReadThreadSafe(RecordHeader header) throws Except startLatch.await(); header.key(); header.value(); - } catch (NullPointerException e) { - raceDetected.set(true); } catch (InterruptedException e) { Thread.currentThread().interrupt(); throw new RuntimeException(e); @@ -294,8 +291,6 @@ private void assertRecordHeaderReadThreadSafe(RecordHeader header) throws Except startLatch.countDown(); futures.forEach(CompletableFuture::join); - - assertFalse(raceDetected.get()); } @RepeatedTest(100) From 808d8db8c86a3512e8e5976256530a80be8ea789 Mon Sep 17 00:00:00 2001 From: Ming-Yen Chung Date: Fri, 31 Oct 2025 23:40:44 +0800 Subject: [PATCH 09/10] Removed unused import --- .../apache/kafka/common/header/internals/RecordHeadersTest.java | 1 - 1 file changed, 1 deletion(-) diff --git a/clients/src/test/java/org/apache/kafka/common/header/internals/RecordHeadersTest.java b/clients/src/test/java/org/apache/kafka/common/header/internals/RecordHeadersTest.java index 56e8b41f903ec..605b5598e133d 100644 --- a/clients/src/test/java/org/apache/kafka/common/header/internals/RecordHeadersTest.java +++ b/clients/src/test/java/org/apache/kafka/common/header/internals/RecordHeadersTest.java @@ -28,7 +28,6 @@ import java.util.Iterator; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; -import java.util.concurrent.atomic.AtomicBoolean; import java.util.stream.Collectors; import java.util.stream.IntStream; From f162297e0d1fa3a666400a84b1841b5be73ca260 Mon Sep 17 00:00:00 2001 From: Ming-Yen Chung Date: Sun, 2 Nov 2025 12:12:14 +0800 Subject: [PATCH 10/10] correct docs --- .../apache/kafka/clients/consumer/ConsumerRecord.java | 10 ++++++++-- docs/upgrade.html | 2 +- 2 files changed, 9 insertions(+), 3 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecord.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecord.java index a4d6c9cd94b45..fa4ff8905427a 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecord.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecord.java @@ -40,8 +40,14 @@ *
* *

- * However, the {@link #headers()} collection and individual header instances are - * read thread-safe, allowing concurrent access for reading without additional synchronization. + * In particular, the {@link #headers()} method returns a mutable collection of headers. If multiple + * threads access or modify these headers concurrently, it may lead to race conditions or inconsistent + * states. It is the responsibility of the user to ensure that multi-threaded access is properly synchronized. + * + *

+ * However, each individual {@link org.apache.kafka.common.header.Header} instance + * is read thread-safe; that is, it is safe for multiple threads to read the same header's key or value concurrently + * as long as no thread modifies it. * *

* Refer to the {@link KafkaConsumer} documentation for more details on multi-threaded consumption and processing strategies. diff --git a/docs/upgrade.html b/docs/upgrade.html index f3eb793f038f3..a1b9bcaab16ca 100644 --- a/docs/upgrade.html +++ b/docs/upgrade.html @@ -27,7 +27,7 @@

Notable changes in 4