Skip to content
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

[improve] Use single buffer for metrics when noUnsafe use #23612

Merged
merged 6 commits into from
Nov 27, 2024
Merged
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@
import io.netty.buffer.CompositeByteBuf;
import io.netty.buffer.PooledByteBufAllocator;
import io.netty.buffer.Unpooled;
import io.netty.util.internal.PlatformDependent;
import io.prometheus.client.Collector;
import java.io.BufferedOutputStream;
import java.io.IOException;
Expand Down Expand Up @@ -364,19 +365,24 @@ protected ByteBuf generateMetrics(List<PrometheusRawMetricsProvider> metricsProv
}
}

private ByteBuf allocateMultipartCompositeDirectBuffer() {
ByteBuf allocateMultipartCompositeDirectBuffer() {
// use composite buffer with pre-allocated buffers to ensure that the pooled allocator can be used
// for allocating the buffers
ByteBufAllocator byteBufAllocator = PulsarByteBufAllocator.DEFAULT;
int chunkSize = resolveChunkSize(byteBufAllocator);
CompositeByteBuf buf = byteBufAllocator.compositeDirectBuffer(
ByteBuf buf;
if (PlatformDependent.hasUnsafe()) {
int chunkSize = resolveChunkSize(byteBufAllocator);
buf = byteBufAllocator.compositeDirectBuffer(
Math.max(MINIMUM_FOR_MAX_COMPONENTS, (initialBufferSize / chunkSize) + 1));
int totalLen = 0;
while (totalLen < initialBufferSize) {
totalLen += chunkSize;
// increase the capacity in increments of chunkSize to preallocate the buffers
// in the composite buffer
buf.capacity(totalLen);
int totalLen = 0;
while (totalLen < initialBufferSize) {
totalLen += chunkSize;
// increase the capacity in increments of chunkSize to preallocate the buffers
// in the composite buffer
buf.capacity(totalLen);
}
} else {
buf = byteBufAllocator.directBuffer(initialBufferSize);
}
return buf;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,13 +32,22 @@
import java.util.Random;
import java.util.concurrent.ExecutionException;
import java.util.zip.GZIPInputStream;
import lombok.Cleanup;
import org.apache.commons.io.IOUtils;
import org.apache.pulsar.broker.PulsarService;
import org.apache.pulsar.broker.ServiceConfiguration;
import org.apache.pulsar.common.util.SimpleTextOutputStream;
import org.testng.annotations.BeforeClass;
import org.testng.annotations.Test;

public class PrometheusMetricsGeneratorTest {


@BeforeClass
public static void setProperties() {
System.setProperty("io.netty.noUnsafe", "true");
lhotari marked this conversation as resolved.
Show resolved Hide resolved
}

// reproduce issue #22575
@Test
public void testReproducingBufferOverflowExceptionAndEOFExceptionBugsInGzipCompression()
Expand Down Expand Up @@ -82,4 +91,22 @@ protected ByteBuf generateMetrics(List<PrometheusRawMetricsProvider> metricsProv
metricsBuffer.release();
}
}
}

@Test
public void testWriteStringWithNoUnsafe() {
PrometheusMetricsGenerator generator = new PrometheusMetricsGenerator(null, false, false, false, false,
Clock.systemUTC());
@Cleanup("release")
ByteBuf buf = generator.allocateMultipartCompositeDirectBuffer();
for (int i = 0; i < 2; i++) {
buf.writeBytes(new byte[1024 * 1024]);
}

SimpleTextOutputStream outputStream = new SimpleTextOutputStream(buf);
try {
outputStream.write("trest");
} catch (Exception e) {

}
}
}
Loading