Skip to content

[fix][proxy] Fix proxy OOM by replacing TopicName with a simple conversion method #24465

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

Open
wants to merge 5 commits into
base: master
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from 2 commits
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 @@ -55,11 +55,17 @@ public interface MessageProcessor {
void process(RawMessage message) throws IOException;
}

@Deprecated
public static void parseMessage(TopicName topicName, long ledgerId, long entryId, ByteBuf headersAndPayload,
MessageProcessor processor, int maxMessageSize) throws IOException {
parseMessage(topicName.toString(), ledgerId, entryId, headersAndPayload, processor, maxMessageSize);
}

/**
* Parse a raw Pulsar entry payload and extract all the individual message that may be included in the batch. The
* provided {@link MessageProcessor} will be invoked for each individual message.
*/
public static void parseMessage(TopicName topicName, long ledgerId, long entryId, ByteBuf headersAndPayload,
public static void parseMessage(String topicName, long ledgerId, long entryId, ByteBuf headersAndPayload,
MessageProcessor processor, int maxMessageSize) throws IOException {
ByteBuf payload = headersAndPayload;
ByteBuf uncompressedPayload = null;
Expand Down Expand Up @@ -117,7 +123,7 @@ public static void parseMessage(TopicName topicName, long ledgerId, long entryId
}
}

public static boolean verifyChecksum(TopicName topic, ByteBuf headersAndPayload, long ledgerId, long entryId) {
public static boolean verifyChecksum(String topic, ByteBuf headersAndPayload, long ledgerId, long entryId) {
if (hasChecksum(headersAndPayload)) {
int checksum = readChecksum(headersAndPayload);
int computedChecksum = computeChecksum(headersAndPayload);
Expand All @@ -132,7 +138,14 @@ public static boolean verifyChecksum(TopicName topic, ByteBuf headersAndPayload,
return true;
}

public static ByteBuf uncompressPayloadIfNeeded(TopicName topic, MessageMetadata msgMetadata,
@Deprecated
public static ByteBuf uncompressPayloadIfNeeded(TopicName topicName, MessageMetadata msgMetadata,
ByteBuf payload, long ledgerId, long entryId, int maxMessageSize) {
return uncompressPayloadIfNeeded(topicName.toString(), msgMetadata, payload, ledgerId, entryId,
maxMessageSize);
}

public static ByteBuf uncompressPayloadIfNeeded(String topic, MessageMetadata msgMetadata,
ByteBuf payload, long ledgerId, long entryId, int maxMessageSize) {
CompressionCodec codec = CompressionCodecProvider.getCompressionCodec(msgMetadata.getCompression());
int uncompressedSize = msgMetadata.getUncompressedSize();
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,104 @@
/*
* 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.pulsar.common.naming;

import java.util.ArrayList;
import java.util.List;
import org.apache.commons.lang3.StringUtils;

/**
* {@link TopicName} is over heavy in many simple cases. This util class provides many methods to perform topic name
* conversions quickly without a global cache.
*/
public class TopicNameUtils {

/**
* Convert a topic name to a full topic name.
* In Pulsar, a full topic name is "<domain>://<tenant>/<namespace>/<local-topic>" (v2) or
* "<domain>://<tenant>/<cluster>/<namespace>/<local-topic>" (v1). However, for convenient, it's allowed for clients
* to pass a short topic name with v2 format:
* - "<local-topic>", which represents "persistent://public/default/<local-topic>"
* - "<tenant>/<namespace>/<local-topic>, which represents "persistent://<tenant>/<namespace>/<local-topic>"
*
* @param topic the topic name from client
* @return the full topic name.
*/
public static String toFullTopicName(String topic) {
final int index = topic.indexOf("://");
if (index >= 0) {
TopicDomain.getEnum(topic.substring(0, index));
final List<String> parts = splitBySlash(topic, 4);
if (parts.size() != 3 && parts.size() != 4) {
throw new IllegalArgumentException(topic + " is invalid");
}
if (parts.size() == 3) {
NamespaceName.validateNamespaceName(parts.get(0), parts.get(1));
if (StringUtils.isBlank(parts.get(2))) {
throw new IllegalArgumentException(topic + " has blank local topic");
}
} else {
NamespaceName.validateNamespaceName(parts.get(0), parts.get(1), parts.get(2));
if (StringUtils.isBlank(parts.get(3))) {
throw new IllegalArgumentException(topic + " has blank local topic");
}
}
return topic; // it's a valid full topic name
} else {
List<String> parts = splitBySlash(topic, 0);
if (parts.size() != 1 && parts.size() != 3) {
throw new IllegalArgumentException(topic + " is invalid");
}
if (parts.size() == 1) {
if (StringUtils.isBlank(parts.get(0))) {
throw new IllegalArgumentException(topic + " has blank local topic");
}
return "persistent://public/default/" + parts.get(0);
} else {
NamespaceName.validateNamespaceName(parts.get(0), parts.get(1));
if (StringUtils.isBlank(parts.get(2))) {
throw new IllegalArgumentException(topic + " has blank local topic");
}
return "persistent://" + topic;
}
}
}

private static List<String> splitBySlash(String topic, int limit) {
final List<String> tokens = new ArrayList<>(3);
final int loopCount = (limit <= 0) ? Integer.MAX_VALUE : limit - 1;
int beginIndex = 0;
for (int i = 0; i < loopCount; i++) {
final int endIndex = topic.indexOf('/', beginIndex);
if (endIndex < 0) {
tokens.add(topic.substring(beginIndex));
return tokens;
} else if (endIndex > beginIndex) {
tokens.add(topic.substring(beginIndex, endIndex));
} else {
throw new IllegalArgumentException("Invalid topic name " + topic);
}
beginIndex = endIndex + 1;
}
if (beginIndex >= topic.length()) {
throw new IllegalArgumentException("Invalid topic name " + topic);
}
tokens.add(topic.substring(beginIndex));
return tokens;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
import static org.testng.Assert.assertFalse;
import static org.testng.Assert.assertNotEquals;
import static org.testng.Assert.assertNull;
import static org.testng.Assert.assertThrows;
import static org.testng.Assert.assertTrue;
import static org.testng.Assert.fail;
import org.apache.pulsar.common.util.Codec;
Expand Down Expand Up @@ -52,9 +53,8 @@ public void topic() {

assertEquals(TopicName.get("persistent://tenant/cluster/namespace/topic").toString(),
"persistent://tenant/cluster/namespace/topic");

assertNotEquals(TopicName.get("persistent://tenant/cluster/namespace/topic"),
"persistent://tenant/cluster/namespace/topic");
assertEquals(TopicNameUtils.toFullTopicName("persistent://tenant/cluster/namespace/topic"),
"persistent://tenant/cluster/namespace/topic");

assertEquals(TopicName.get("persistent://tenant/cluster/namespace/topic").getDomain(),
TopicDomain.persistent);
Expand Down Expand Up @@ -103,62 +103,79 @@ public void topic() {
} catch (IllegalArgumentException e) {
// Ok
}
assertThrows(IllegalArgumentException.class,
() -> TopicNameUtils.toFullTopicName("://tenant.namespace:my-topic"));

try {
TopicName.get("://tenant.namespace");
fail("Should have raised exception");
} catch (IllegalArgumentException e) {
// Ok
}
assertThrows(IllegalArgumentException.class, () -> TopicNameUtils.toFullTopicName("://tenant.namespace"));

try {
TopicName.get("invalid://tenant/cluster/namespace/topic");
fail("Should have raied exception");
} catch (IllegalArgumentException e) {
// Ok
}
assertThrows(IllegalArgumentException.class,
() -> TopicNameUtils.toFullTopicName("invalid://tenant/cluster/namespace/topic"));

try {
TopicName.get("tenant/cluster/namespace/topic");
fail("Should have raised exception");
} catch (IllegalArgumentException e) {
// Ok
}
assertThrows(IllegalArgumentException.class,
() -> TopicNameUtils.toFullTopicName("tenant/cluster/namespace/topic"));

try {
TopicName.get("persistent:///cluster/namespace/mydest-1");
fail("Should have raised exception");
} catch (IllegalArgumentException e) {
// Ok
}
assertThrows(IllegalArgumentException.class,
() -> TopicNameUtils.toFullTopicName("persistent:///cluster/namespace/mydest-1"));

try {
TopicName.get("persistent://pulsar//namespace/mydest-1");
fail("Should have raised exception");
} catch (IllegalArgumentException e) {
// Ok
}
assertThrows(IllegalArgumentException.class,
() -> TopicNameUtils.toFullTopicName("persistent://pulsar//namespace/mydest-1"));

try {
TopicName.get("persistent://pulsar/cluster//mydest-1");
fail("Should have raised exception");
} catch (IllegalArgumentException e) {
// Ok
}
assertThrows(IllegalArgumentException.class,
() -> TopicNameUtils.toFullTopicName("persistent://pulsar/cluster//mydest-1"));

try {
TopicName.get("persistent://pulsar/cluster/namespace/");
fail("Should have raised exception");
} catch (IllegalArgumentException e) {
// Ok
}
assertThrows(IllegalArgumentException.class,
() -> TopicNameUtils.toFullTopicName("persistent://pulsar/cluster/namespace/"));

try {
TopicName.get("://pulsar/cluster/namespace/");
fail("Should have raised exception");
} catch (IllegalArgumentException e) {
// Ok
}
assertThrows(IllegalArgumentException.class,
() -> TopicNameUtils.toFullTopicName("://pulsar/cluster/namespace/"));

assertEquals(TopicName.get("persistent://tenant/cluster/namespace/topic")
.getPersistenceNamingEncoding(), "tenant/cluster/namespace/persistent/topic");
Expand All @@ -169,20 +186,24 @@ public void topic() {
} catch (IllegalArgumentException e) {
// Ok
}
assertThrows(IllegalArgumentException.class, () -> TopicNameUtils.toFullTopicName("://tenant.namespace"));

try {
TopicName.get("://tenant/cluster/namespace");
fail("Should have raied exception");
} catch (IllegalArgumentException e) {
// Ok
}
assertThrows(IllegalArgumentException.class,
() -> TopicNameUtils.toFullTopicName("://tenant//cluster/namespace"));

try {
TopicName.get(" ");
fail("Should have raised exception");
} catch (IllegalArgumentException e) {
// Ok
}
assertThrows(IllegalArgumentException.class, () -> TopicNameUtils.toFullTopicName(" "));

TopicName nameWithSlash = TopicName.get("persistent://tenant/cluster/namespace/ns-abc/table/1");
assertEquals(nameWithSlash.getEncodedLocalName(), Codec.encode("ns-abc/table/1"));
Expand Down Expand Up @@ -344,4 +365,14 @@ public void testTwoKeyWordPartition(){
assertNotEquals(tp2.toString(), tp1.toString());
assertEquals(tp2.toString(), "persistent://tenant1/namespace1/tp1-partition-0-DLQ-partition-0");
}

@Test
public void testToFullTopicName() {
// There is no constraint for local topic name
assertEquals("persistent://public/default/tp???xx=", TopicNameUtils.toFullTopicName("tp???xx="));
assertEquals("persistent://tenant/ns/tp???xx=", TopicNameUtils.toFullTopicName("tenant/ns/tp???xx="));
assertThrows(IllegalArgumentException.class, () -> TopicNameUtils.toFullTopicName("ns/topic"));
// v1 format is not supported when the domain is not included
assertThrows(IllegalArgumentException.class, () -> TopicNameUtils.toFullTopicName("tenant/cluster/ns/topic"));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,8 @@
import java.net.SocketAddress;
import java.net.URI;
import java.net.URISyntaxException;
import java.util.HashMap;
import java.util.Map;
import java.util.Optional;
import java.util.concurrent.Semaphore;
import org.apache.commons.lang3.StringUtils;
Expand All @@ -36,7 +38,7 @@
import org.apache.pulsar.common.api.proto.CommandLookupTopicResponse.LookupType;
import org.apache.pulsar.common.api.proto.CommandPartitionedTopicMetadata;
import org.apache.pulsar.common.api.proto.ServerError;
import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.naming.TopicNameUtils;
import org.apache.pulsar.common.protocol.Commands;
import org.apache.pulsar.common.protocol.schema.BytesSchemaVersion;
import org.apache.pulsar.common.protocol.schema.SchemaVersion;
Expand Down Expand Up @@ -85,6 +87,9 @@ public class LookupProxyHandler {
.create().register();
private final Semaphore lookupRequestSemaphore;

// Maps the topic name from the request to the full topic name
private final Map<String, String> topicNameCache = new HashMap<>();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

When multiple connections access the same topic, will it cause more heap memory occupation?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes. However, once a connection is disconnected, the cache will be garbage collected. In extreme cases, if many consumers subscribe

How about removing the cache? This conversion is actually very fast. It's only about 2.2x slower than reading from a TopicName cache.

TopicNameBenchmark.testConvert        thrpt       284.114          ops/s
TopicNameBenchmark.testReadFromCache  thrpt       644.718          ops/s

and it can execute 30000*284 operations in 1 second.

image

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

i.e. each operation only spends only 0.12 MICROSECONDS. The "space-for-time tradeoff" might not worth here.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I removed the cache and updated the PR description

Copy link
Contributor Author

@BewareMyPower BewareMyPower Jun 26, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

image

Well, if there are 500k keys, TopicName#get is slower.

TopicNameBenchmark.testConvert        thrpt       5.857          ops/s
TopicNameBenchmark.testReadFromCache  thrpt       5.326          ops/s

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

200k keys:

TopicNameBenchmark.testConvert        thrpt       14.706          ops/s
TopicNameBenchmark.testReadFromCache  thrpt       16.639          ops/s


public LookupProxyHandler(ProxyService proxy, ProxyConnection proxyConnection) {
this.discoveryProvider = proxy.getDiscoveryProvider();
this.lookupRequestSemaphore = proxy.getLookupRequestSemaphore();
Expand Down Expand Up @@ -221,7 +226,8 @@ public void handlePartitionMetadataResponse(CommandPartitionedTopicMetadata part
**/
private void handlePartitionMetadataResponse(CommandPartitionedTopicMetadata partitionMetadata,
long clientRequestId) {
TopicName topicName = TopicName.get(partitionMetadata.getTopic());
String topicName = topicNameCache.computeIfAbsent(partitionMetadata.getTopic(),
TopicNameUtils::toFullTopicName);

String serviceUrl = getBrokerServiceUrl(clientRequestId);
if (serviceUrl == null) {
Expand All @@ -235,7 +241,7 @@ private void handlePartitionMetadataResponse(CommandPartitionedTopicMetadata par

if (log.isDebugEnabled()) {
log.debug("Getting connections to '{}' for Looking up topic '{}' with clientReq Id '{}'", addr,
topicName.getPartitionedTopicName(), clientRequestId);
topicName, clientRequestId);
}
proxyConnection.getConnectionPool().getConnection(addr).thenAccept(clientCnx -> {
// Connected to backend broker
Expand All @@ -245,7 +251,7 @@ private void handlePartitionMetadataResponse(CommandPartitionedTopicMetadata par
partitionMetadata.isMetadataAutoCreationEnabled());
clientCnx.newLookup(command, requestId).whenComplete((r, t) -> {
if (t != null) {
log.warn("[{}] failed to get Partitioned metadata : {}", topicName.toString(),
log.warn("[{}] failed to get Partitioned metadata : {}", topicName,
t.getMessage(), t);
PulsarClientException pce = PulsarClientException.unwrap(t);
writeAndFlush(Commands.newLookupErrorResponse(clientCnx.revertClientExToErrorCode(pce),
Expand Down
Loading
Loading