From 7071ded2a61448da21c149fa1d85b3999b0d2f73 Mon Sep 17 00:00:00 2001
From: Guozhang Wang
Date: Tue, 23 Mar 2021 21:05:42 -0700
Subject: [PATCH] KAFKA-12524: Remove deprecated segments() (#10379)
Reviewers: Boyang Chen
---
docs/streams/upgrade-guide.html | 8 ++++++++
.../kafka/streams/state/WindowBytesStoreSupplier.java | 10 ----------
.../internals/InMemoryWindowBytesStoreSupplier.java | 6 ------
.../internals/RocksDbWindowBytesStoreSupplier.java | 6 ------
4 files changed, 8 insertions(+), 22 deletions(-)
diff --git a/docs/streams/upgrade-guide.html b/docs/streams/upgrade-guide.html
index 72ec4e0b96b..fbafebe967e 100644
--- a/docs/streams/upgrade-guide.html
+++ b/docs/streams/upgrade-guide.html
@@ -92,6 +92,14 @@
this restriction will be enforced. If you wish to run more than one instance of Kafka Streams, you must configure them with different values for state.dir
.
+
+
+ We removed the following deprecated APIs:
+
+
+ -
WindowBytesStoreSupplier#segments
: deprecated in Kafka 2.1.0 (KIP-319).
+
+
We extended StreamJoined
to include the options withLoggingEnabled()
and withLoggingDisabled()
in
diff --git a/streams/src/main/java/org/apache/kafka/streams/state/WindowBytesStoreSupplier.java b/streams/src/main/java/org/apache/kafka/streams/state/WindowBytesStoreSupplier.java
index 10b96e682d1..9ced28c1800 100644
--- a/streams/src/main/java/org/apache/kafka/streams/state/WindowBytesStoreSupplier.java
+++ b/streams/src/main/java/org/apache/kafka/streams/state/WindowBytesStoreSupplier.java
@@ -27,16 +27,6 @@ import org.apache.kafka.common.utils.Bytes;
* 2. Null value bytes should never be returned in range query results.
*/
public interface WindowBytesStoreSupplier extends StoreSupplier> {
- /**
- * The number of segments the store has. If your store is segmented then this should be the number of segments
- * in the underlying store.
- * It is also used to reduce the amount of data that is scanned when caching is enabled.
- *
- * @return number of segments
- * @deprecated since 2.1. Use {@link WindowBytesStoreSupplier#segmentIntervalMs()} instead.
- */
- @Deprecated
- int segments();
/**
* The size of the segments (in milliseconds) the store has.
diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryWindowBytesStoreSupplier.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryWindowBytesStoreSupplier.java
index 52a2cd84e71..4ab93ec36cb 100644
--- a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryWindowBytesStoreSupplier.java
+++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryWindowBytesStoreSupplier.java
@@ -55,12 +55,6 @@ public class InMemoryWindowBytesStoreSupplier implements WindowBytesStoreSupplie
return "in-memory-window";
}
- @Deprecated
- @Override
- public int segments() {
- throw new IllegalStateException("Segments is deprecated and should not be called");
- }
-
@Override
public long retentionPeriod() {
return retentionPeriod;
diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDbWindowBytesStoreSupplier.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDbWindowBytesStoreSupplier.java
index 9ce58ba3585..5ccdd7a1746 100644
--- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDbWindowBytesStoreSupplier.java
+++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDbWindowBytesStoreSupplier.java
@@ -77,12 +77,6 @@ public class RocksDbWindowBytesStoreSupplier implements WindowBytesStoreSupplier
return "rocksdb-window";
}
- @Deprecated
- @Override
- public int segments() {
- return (int) (retentionPeriod / segmentInterval) + 1;
- }
-
@Override
public long segmentIntervalMs() {
return segmentInterval;