-
Notifications
You must be signed in to change notification settings - Fork 56
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Log the sizes allocated through the SimpleMemoryPool for analysis. #380
base: 3.0-li
Are you sure you want to change the base?
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,97 @@ | ||
/* | ||
* 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.kafka.common.memory; | ||
|
||
import java.util.HashMap; | ||
import java.util.Map; | ||
import java.util.concurrent.atomic.AtomicInteger; | ||
import org.slf4j.Logger; | ||
import org.slf4j.LoggerFactory; | ||
|
||
|
||
public class MemoryPoolStatsStore { | ||
private static final Logger log = LoggerFactory.getLogger(MemoryPoolStatsStore.class); | ||
|
||
private final AtomicInteger[] histogram; | ||
private final int maxSizeBytes; | ||
private final int segmentSizeBytes; | ||
|
||
public static class Range { | ||
public final int startInclusive; | ||
public final int endInclusive; | ||
|
||
public Range(int startInclusive, int endInclusive) { | ||
this.startInclusive = startInclusive; | ||
this.endInclusive = endInclusive; | ||
} | ||
|
||
@Override | ||
public String toString() { | ||
return "Range{" + "startInclusive=" + startInclusive + ", endInclusive=" + endInclusive + '}'; | ||
} | ||
} | ||
|
||
public MemoryPoolStatsStore(int segments, int maxSizeBytes) { | ||
histogram = new AtomicInteger[segments]; | ||
this.maxSizeBytes = maxSizeBytes; | ||
segmentSizeBytes = (int) Math.ceil((double) maxSizeBytes / segments); | ||
for (int segmentIndex = 0; segmentIndex < segments; segmentIndex++) { | ||
histogram[segmentIndex] = new AtomicInteger(); | ||
} | ||
} | ||
|
||
private int getSegmentIndexForBytes(int bytes) { | ||
if (bytes == 0) { | ||
throw new IllegalArgumentException("Requested zero bytes for allocation."); | ||
} | ||
if (bytes > maxSizeBytes) { | ||
log.debug("Requested bytes {} for allocation exceeds maximum recorded value {}", bytes, maxSizeBytes); | ||
return -1; | ||
} else { | ||
return (bytes - 1) / segmentSizeBytes; | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Would it be better to record the bytes in log scale instead of linear scale? |
||
} | ||
} | ||
|
||
public void recordAllocation(int bytes) { | ||
try { | ||
final int segmentIndex = getSegmentIndexForBytes(bytes); | ||
if (segmentIndex != -1) { | ||
histogram[segmentIndex].incrementAndGet(); | ||
} | ||
} catch (IllegalArgumentException e) { | ||
log.error("Encountered error when trying to record memory allocation for request", e); | ||
} | ||
} | ||
|
||
public synchronized Map<Range, Integer> getFrequencies() { | ||
Map<Range, Integer> frequenciesMap = new HashMap<>(); | ||
for (int segmentIndex = 0; segmentIndex < histogram.length; segmentIndex++) { | ||
frequenciesMap.put(new Range( | ||
segmentIndex * segmentSizeBytes + 1, | ||
segmentIndex * segmentSizeBytes + segmentSizeBytes | ||
), histogram[segmentIndex].intValue()); | ||
} | ||
return frequenciesMap; | ||
} | ||
|
||
public synchronized void clear() { | ||
for (AtomicInteger atomicInteger : histogram) { | ||
atomicInteger.set(0); | ||
} | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -17,6 +17,7 @@ | |
package org.apache.kafka.common.memory; | ||
|
||
import java.nio.ByteBuffer; | ||
import java.util.Optional; | ||
import java.util.concurrent.atomic.AtomicLong; | ||
|
||
import org.apache.kafka.common.metrics.Sensor; | ||
|
@@ -37,10 +38,13 @@ public class SimpleMemoryPool implements MemoryPool { | |
protected final AtomicLong availableMemory; | ||
protected final int maxSingleAllocationSize; | ||
protected final AtomicLong startOfNoMemPeriod = new AtomicLong(); //nanoseconds | ||
private final Optional<MemoryPoolStatsStore> memoryPoolStatsStore; | ||
protected volatile Sensor oomTimeSensor; | ||
protected volatile Sensor allocateSensor; | ||
|
||
public SimpleMemoryPool(long sizeInBytes, int maxSingleAllocationBytes, boolean strict, Sensor oomPeriodSensor, Sensor allocateSensor) { | ||
public SimpleMemoryPool(long sizeInBytes, int maxSingleAllocationBytes, boolean strict, Sensor oomPeriodSensor, | ||
Sensor allocateSensor, Optional<MemoryPoolStatsStore> memoryPoolStatsStore) { | ||
this.memoryPoolStatsStore = memoryPoolStatsStore; | ||
if (sizeInBytes <= 0 || maxSingleAllocationBytes <= 0 || maxSingleAllocationBytes > sizeInBytes) | ||
throw new IllegalArgumentException("must provide a positive size and max single allocation size smaller than size." | ||
+ "provided " + sizeInBytes + " and " + maxSingleAllocationBytes + " respectively"); | ||
|
@@ -57,7 +61,8 @@ public ByteBuffer tryAllocate(int sizeBytes) { | |
if (sizeBytes < 1) | ||
throw new IllegalArgumentException("requested size " + sizeBytes + "<=0"); | ||
if (sizeBytes > maxSingleAllocationSize) | ||
throw new IllegalArgumentException("requested size " + sizeBytes + " is larger than maxSingleAllocationSize " + maxSingleAllocationSize); | ||
throw new IllegalArgumentException( | ||
"requested size " + sizeBytes + " is larger than maxSingleAllocationSize " + maxSingleAllocationSize); | ||
|
||
long available; | ||
boolean success = false; | ||
|
@@ -114,6 +119,7 @@ public boolean isOutOfMemory() { | |
//allows subclasses to do their own bookkeeping (and validation) _before_ memory is returned to client code. | ||
protected void bufferToBeReturned(ByteBuffer justAllocated) { | ||
this.allocateSensor.record(justAllocated.capacity()); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Is the metrics from this allocateSensor not good enough? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This sensor relies on the |
||
memoryPoolStatsStore.ifPresent(sizeStore -> sizeStore.recordAllocation(justAllocated.capacity())); | ||
log.trace("allocated buffer of size {} ", justAllocated.capacity()); | ||
} | ||
|
||
|
Original file line number | Diff line number | Diff line change | ||
---|---|---|---|---|
|
@@ -37,6 +37,7 @@ import kafka.utils._ | |||
import kafka.zk.{AdminZkClient, BrokerInfo, KafkaZkClient} | ||||
import org.apache.kafka.clients.{ApiVersions, ManualMetadataUpdater, NetworkClient, NetworkClientUtils} | ||||
import org.apache.kafka.common.internals.Topic | ||||
import org.apache.kafka.common.memory.MemoryPoolStatsStore | ||||
import org.apache.kafka.common.message.ApiMessageType.ListenerType | ||||
import org.apache.kafka.common.message.ControlledShutdownRequestData | ||||
import org.apache.kafka.common.metrics.Metrics | ||||
|
@@ -46,13 +47,14 @@ import org.apache.kafka.common.requests.{ControlledShutdownRequest, ControlledSh | |||
import org.apache.kafka.common.security.scram.internals.ScramMechanism | ||||
import org.apache.kafka.common.security.token.delegation.internals.DelegationTokenCache | ||||
import org.apache.kafka.common.security.{JaasContext, JaasUtils} | ||||
import org.apache.kafka.common.utils.{AppInfoParser, LogContext, Time, Utils, PoisonPill} | ||||
import org.apache.kafka.common.utils.{AppInfoParser, LogContext, PoisonPill, Time, Utils} | ||||
import org.apache.kafka.common.{Endpoint, Node, TopicPartition} | ||||
import org.apache.kafka.metadata.BrokerState | ||||
import org.apache.kafka.server.authorizer.Authorizer | ||||
import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig | ||||
import org.apache.zookeeper.client.ZKClientConfig | ||||
|
||||
import java.util.Optional | ||||
import scala.collection.{Map, Seq} | ||||
import scala.jdk.CollectionConverters._ | ||||
import scala.collection.mutable.{ArrayBuffer, Buffer} | ||||
|
@@ -323,13 +325,43 @@ class KafkaServer( | |||
|
||||
observer = Observer(config) | ||||
|
||||
def initializeMemoryPoolStats(): Optional[MemoryPoolStatsStore] = { | ||||
if (!config.memoryPoolStatsLoggingEnable) { | ||||
return Optional.empty() | ||||
} | ||||
|
||||
info(s"Memory pool stats logging is enabled, segments = " + | ||||
s"${config.memoryPoolStatsNumSegments}, max size = ${config.memoryPoolStatsMaxSize}, " + | ||||
s"logging frequency in minutes = ${config.memoryPoolStatsLoggingFrequencyMinutes}") | ||||
val memoryPoolStatsStore = new MemoryPoolStatsStore(config.memoryPoolStatsNumSegments, config.memoryPoolStatsMaxSize) | ||||
val requestStatsLogger = new MemoryPoolStatsLogger() | ||||
|
||||
def publishHistogramToLog(): Unit = { | ||||
info("Publishing memory pool stats") | ||||
requestStatsLogger.logStats(memoryPoolStatsStore) | ||||
memoryPoolStatsStore.clear() | ||||
} | ||||
|
||||
val histogramPublisher = new KafkaScheduler(threads = 1, "histogram-publisher-") | ||||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Can we reuse the same scheduler
since the startup and shutting down of that scheduler is already taken care of. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Yeah we can do that. I was not sure myself, because we use this I have fixed it. |
||||
histogramPublisher.startup() | ||||
histogramPublisher.schedule(name = "publish-histogram-to-log", | ||||
fun = publishHistogramToLog, | ||||
period = config.memoryPoolStatsLoggingFrequencyMinutes.toLong, | ||||
unit = TimeUnit.MINUTES) | ||||
|
||||
Optional.of(memoryPoolStatsStore) | ||||
} | ||||
|
||||
val memoryPoolStatsStore = initializeMemoryPoolStats() | ||||
|
||||
// Create and start the socket server acceptor threads so that the bound port is known. | ||||
// Delay starting processors until the end of the initialization sequence to ensure | ||||
// that credentials have been loaded before processing authentications. | ||||
// | ||||
// Note that we allow the use of KRaft mode controller APIs when forwarding is enabled | ||||
// so that the Envelope request is exposed. This is only used in testing currently. | ||||
socketServer = new SocketServer(config, metrics, time, credentialProvider, observer, apiVersionManager) | ||||
socketServer = new SocketServer( | ||||
config, metrics, time, credentialProvider, observer, apiVersionManager, memoryPoolStatsStore) | ||||
socketServer.startup(startProcessingRequests = false) | ||||
|
||||
/* start replica manager */ | ||||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Is there a reason to not use the metrics object but implement this by hand?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yes. Kafka metrics support publishing histograms only as percentiles. This will not be useful information for us to determine the range of most frequent memory allocations. For example, consider that around 60% of requests are of size 1 kb. Now, if we observe from metrics that the p90 request size is 400k and the p99 size is 1500k, we are unable to make any inference about the fact that most requests are within 1 kb range.
Hence, we are publishing the full histogram as logs.