Skip to content

Commit a6c53d0

Browse files
KAFKA-18878: Added share session cache and delayed share fetch metrics (KIP-1103) (apache#19059)
The PR implements the ShareSessionCache and DelayedShareFetchMetrics as defined in KIP-1103. Reviewers: Andrew Schofield <[email protected]>
1 parent ff94c44 commit a6c53d0

File tree

6 files changed

+162
-29
lines changed

6 files changed

+162
-29
lines changed

core/src/main/java/kafka/server/share/DelayedShareFetch.java

Lines changed: 19 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -25,6 +25,7 @@
2525
import org.apache.kafka.common.protocol.Errors;
2626
import org.apache.kafka.common.requests.FetchRequest;
2727
import org.apache.kafka.common.utils.Time;
28+
import org.apache.kafka.server.metrics.KafkaMetricsGroup;
2829
import org.apache.kafka.server.purgatory.DelayedOperation;
2930
import org.apache.kafka.server.share.SharePartitionKey;
3031
import org.apache.kafka.server.share.fetch.DelayedShareFetchGroupKey;
@@ -36,6 +37,8 @@
3637
import org.apache.kafka.storage.internals.log.LogOffsetMetadata;
3738
import org.apache.kafka.storage.internals.log.LogOffsetSnapshot;
3839

40+
import com.yammer.metrics.core.Meter;
41+
3942
import org.slf4j.Logger;
4043
import org.slf4j.LoggerFactory;
4144

@@ -45,6 +48,7 @@
4548
import java.util.Map;
4649
import java.util.Optional;
4750
import java.util.Set;
51+
import java.util.concurrent.TimeUnit;
4852
import java.util.concurrent.locks.Lock;
4953
import java.util.function.BiConsumer;
5054
import java.util.stream.Collectors;
@@ -61,6 +65,8 @@ public class DelayedShareFetch extends DelayedOperation {
6165

6266
private static final Logger log = LoggerFactory.getLogger(DelayedShareFetch.class);
6367

68+
private static final String EXPIRES_PER_SEC = "ExpiresPerSec";
69+
6470
private final ShareFetch shareFetch;
6571
private final ReplicaManager replicaManager;
6672
private final BiConsumer<SharePartitionKey, Throwable> exceptionHandler;
@@ -70,6 +76,10 @@ public class DelayedShareFetch extends DelayedOperation {
7076
// The topic partitions that need to be completed for the share fetch request are given by sharePartitions.
7177
// sharePartitions is a subset of shareFetchData. The order of insertion/deletion of entries in sharePartitions is important.
7278
private final LinkedHashMap<TopicIdPartition, SharePartition> sharePartitions;
79+
/**
80+
* Metric for the rate of expired delayed fetch requests.
81+
*/
82+
private final Meter expiredRequestMeter;
7383
// Tracks the start time to acquire any share partition for a fetch request.
7484
private long acquireStartTimeMs;
7585
private LinkedHashMap<TopicIdPartition, Long> partitionsAcquired;
@@ -124,10 +134,14 @@ public DelayedShareFetch(
124134
this.shareGroupMetrics = shareGroupMetrics;
125135
this.time = time;
126136
this.acquireStartTimeMs = time.hiResClockMs();
137+
// Register metrics for DelayedShareFetch.
138+
KafkaMetricsGroup metricsGroup = new KafkaMetricsGroup("kafka.server", "DelayedShareFetchMetrics");
139+
this.expiredRequestMeter = metricsGroup.newMeter(EXPIRES_PER_SEC, "requests", TimeUnit.SECONDS);
127140
}
128141

129142
@Override
130143
public void onExpiration() {
144+
expiredRequestMeter.mark();
131145
}
132146

133147
/**
@@ -514,4 +528,9 @@ void releasePartitionLocks(Set<TopicIdPartition> topicIdPartitions) {
514528
Lock lock() {
515529
return lock;
516530
}
531+
532+
// Visible for testing.
533+
Meter expiredRequestMeter() {
534+
return expiredRequestMeter;
535+
}
517536
}

core/src/test/java/kafka/server/share/DelayedShareFetchTest.java

Lines changed: 18 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -154,6 +154,7 @@ public void testDelayedShareFetchTryCompleteReturnsFalseDueToNonAcquirablePartit
154154
// Metrics shall not be recorded as no partition is acquired.
155155
assertNull(shareGroupMetrics.topicPartitionsAcquireTimeMs(groupId));
156156
assertNull(shareGroupMetrics.topicPartitionsFetchRatio(groupId));
157+
assertEquals(0, delayedShareFetch.expiredRequestMeter().count());
157158

158159
delayedShareFetch.lock().unlock();
159160
}
@@ -1118,6 +1119,23 @@ public void testPartitionMaxBytesFromUniformStrategyInCombineLogReadResponse() {
11181119
true);
11191120
}
11201121

1122+
@Test
1123+
public void testOnCompleteExecutionOnTimeout() {
1124+
ShareFetch shareFetch = new ShareFetch(
1125+
FETCH_PARAMS, "grp", Uuid.randomUuid().toString(),
1126+
new CompletableFuture<>(), new LinkedHashMap<>(), BATCH_SIZE, MAX_FETCH_RECORDS,
1127+
BROKER_TOPIC_STATS);
1128+
DelayedShareFetch delayedShareFetch = DelayedShareFetchBuilder.builder()
1129+
.withShareFetchData(shareFetch)
1130+
.build();
1131+
assertFalse(delayedShareFetch.isCompleted());
1132+
assertFalse(shareFetch.isCompleted());
1133+
// Call run to execute onComplete and onExpiration.
1134+
delayedShareFetch.run();
1135+
assertTrue(shareFetch.isCompleted());
1136+
assertEquals(1, delayedShareFetch.expiredRequestMeter().count());
1137+
}
1138+
11211139
static void mockTopicIdPartitionToReturnDataEqualToMinBytes(ReplicaManager replicaManager, TopicIdPartition topicIdPartition, int minBytes) {
11221140
LogOffsetMetadata hwmOffsetMetadata = new LogOffsetMetadata(1, 1, minBytes);
11231141
LogOffsetSnapshot endOffsetSnapshot = new LogOffsetSnapshot(1, mock(LogOffsetMetadata.class),

core/src/test/java/kafka/server/share/SharePartitionTest.java

Lines changed: 1 addition & 16 deletions
Original file line numberDiff line numberDiff line change
@@ -48,7 +48,6 @@
4848
import org.apache.kafka.coordinator.group.GroupConfig;
4949
import org.apache.kafka.coordinator.group.GroupConfigManager;
5050
import org.apache.kafka.coordinator.group.ShareGroupAutoOffsetResetStrategy;
51-
import org.apache.kafka.server.metrics.KafkaYammerMetrics;
5251
import org.apache.kafka.server.share.acknowledge.ShareAcknowledgementBatch;
5352
import org.apache.kafka.server.share.fetch.ShareAcquiredRecords;
5453
import org.apache.kafka.server.share.metrics.SharePartitionMetrics;
@@ -67,8 +66,6 @@
6766
import org.apache.kafka.storage.internals.log.OffsetResultHolder;
6867
import org.apache.kafka.test.TestUtils;
6968

70-
import com.yammer.metrics.core.Gauge;
71-
7269
import org.junit.jupiter.api.AfterEach;
7370
import org.junit.jupiter.api.BeforeEach;
7471
import org.junit.jupiter.api.Test;
@@ -91,6 +88,7 @@
9188

9289
import static kafka.server.share.SharePartition.EMPTY_MEMBER_ID;
9390
import static org.apache.kafka.server.share.fetch.ShareFetchTestUtils.memoryRecordsBuilder;
91+
import static org.apache.kafka.server.share.fetch.ShareFetchTestUtils.yammerMetricValue;
9492
import static org.apache.kafka.test.TestUtils.assertFutureThrows;
9593
import static org.junit.jupiter.api.Assertions.assertArrayEquals;
9694
import static org.junit.jupiter.api.Assertions.assertEquals;
@@ -6736,19 +6734,6 @@ public void mockPersisterReadStateMethod(Persister persister) {
67366734
Mockito.when(persister.readState(Mockito.any())).thenReturn(CompletableFuture.completedFuture(readShareGroupStateResult));
67376735
}
67386736

6739-
private Number yammerMetricValue(String name) {
6740-
try {
6741-
Gauge gauge = (Gauge) KafkaYammerMetrics.defaultRegistry().allMetrics().entrySet().stream()
6742-
.filter(e -> e.getKey().getMBeanName().contains(name))
6743-
.findFirst()
6744-
.orElseThrow()
6745-
.getValue();
6746-
return (Number) gauge.value();
6747-
} catch (Exception e) {
6748-
return 0;
6749-
}
6750-
}
6751-
67526737
private static class SharePartitionBuilder {
67536738

67546739
private int defaultAcquisitionLockTimeoutMs = 30000;

server/src/main/java/org/apache/kafka/server/share/session/ShareSessionCache.java

Lines changed: 27 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -20,11 +20,15 @@
2020
import org.apache.kafka.common.Uuid;
2121
import org.apache.kafka.common.requests.ShareRequestMetadata;
2222
import org.apache.kafka.common.utils.ImplicitLinkedHashCollection;
23+
import org.apache.kafka.server.metrics.KafkaMetricsGroup;
2324
import org.apache.kafka.server.share.CachedSharePartition;
2425

26+
import com.yammer.metrics.core.Meter;
27+
2528
import java.util.HashMap;
2629
import java.util.Map;
2730
import java.util.TreeMap;
31+
import java.util.concurrent.TimeUnit;
2832

2933
/**
3034
* Caches share sessions.
@@ -37,6 +41,17 @@
3741
* must never be acquired while an individual ShareSession lock is already held.
3842
*/
3943
public class ShareSessionCache {
44+
// Visible for testing.
45+
static final String SHARE_SESSIONS_COUNT = "ShareSessionsCount";
46+
// Visible for testing.
47+
static final String SHARE_PARTITIONS_COUNT = "SharePartitionsCount";
48+
private static final String SHARE_SESSION_EVICTIONS_PER_SEC = "ShareSessionEvictionsPerSec";
49+
50+
/**
51+
* Metric for the rate of eviction of share sessions.
52+
*/
53+
private final Meter evictionsMeter;
54+
4055
private final int maxEntries;
4156
private final long evictionMs;
4257
private long numPartitions = 0;
@@ -47,9 +62,15 @@ public class ShareSessionCache {
4762
// Maps last used times to sessions.
4863
private final TreeMap<LastUsedKey, ShareSession> lastUsed = new TreeMap<>();
4964

65+
@SuppressWarnings("this-escape")
5066
public ShareSessionCache(int maxEntries, long evictionMs) {
5167
this.maxEntries = maxEntries;
5268
this.evictionMs = evictionMs;
69+
// Register metrics for ShareSessionCache.
70+
KafkaMetricsGroup metricsGroup = new KafkaMetricsGroup("kafka.server", "ShareSessionCache");
71+
metricsGroup.newGauge(SHARE_SESSIONS_COUNT, this::size);
72+
metricsGroup.newGauge(SHARE_PARTITIONS_COUNT, this::totalPartitions);
73+
this.evictionsMeter = metricsGroup.newMeter(SHARE_SESSION_EVICTIONS_PER_SEC, "evictions", TimeUnit.SECONDS);
5374
}
5475

5576
/**
@@ -136,6 +157,7 @@ public synchronized boolean tryEvict(long now) {
136157
} else if (now - lastUsedEntry.getKey().lastUsedMs() > evictionMs) {
137158
ShareSession session = lastUsedEntry.getValue();
138159
remove(session);
160+
evictionsMeter.mark();
139161
return true;
140162
}
141163
return false;
@@ -159,4 +181,9 @@ public synchronized ShareSessionKey maybeCreateSession(String groupId, Uuid memb
159181
}
160182
return null;
161183
}
184+
185+
// Visible for testing.
186+
Meter evictionsMeter() {
187+
return evictionsMeter;
188+
}
162189
}

server/src/test/java/org/apache/kafka/server/share/fetch/ShareFetchTestUtils.java

Lines changed: 31 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -23,8 +23,11 @@
2323
import org.apache.kafka.common.record.MemoryRecords;
2424
import org.apache.kafka.common.record.MemoryRecordsBuilder;
2525
import org.apache.kafka.common.record.TimestampType;
26+
import org.apache.kafka.server.metrics.KafkaYammerMetrics;
2627
import org.apache.kafka.test.TestUtils;
2728

29+
import com.yammer.metrics.core.Gauge;
30+
2831
import java.io.IOException;
2932
import java.nio.ByteBuffer;
3033
import java.util.LinkedHashMap;
@@ -145,4 +148,32 @@ public static ShareAcquiredRecords createShareAcquiredRecords(AcquiredRecords ac
145148
List.of(acquiredRecords), (int) (acquiredRecords.lastOffset() - acquiredRecords.firstOffset() + 1)
146149
);
147150
}
151+
152+
/**
153+
* Fetch the gauge value from the yammer metrics.
154+
*
155+
* @param name The name of the metric.
156+
* @return The gauge value as a number.
157+
*/
158+
public static Number yammerMetricValue(String name) {
159+
try {
160+
Gauge gauge = (Gauge) KafkaYammerMetrics.defaultRegistry().allMetrics().entrySet().stream()
161+
.filter(e -> e.getKey().getMBeanName().contains(name))
162+
.findFirst()
163+
.orElseThrow()
164+
.getValue();
165+
return (Number) gauge.value();
166+
} catch (Exception e) {
167+
return 0;
168+
}
169+
}
170+
171+
/**
172+
* Clear all the yammer metrics.
173+
*/
174+
public static void clearYammerMetrics() {
175+
KafkaYammerMetrics.defaultRegistry().allMetrics().keySet().forEach(
176+
metricName -> KafkaYammerMetrics.defaultRegistry().removeMetric(metricName)
177+
);
178+
}
148179
}

0 commit comments

Comments
 (0)