Skip to content
Merged
Show file tree
Hide file tree
Changes from all 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
Expand Up @@ -21,6 +21,7 @@
import com.google.common.base.Splitter;
import com.google.re2j.Pattern;
import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
import java.util.ArrayList;
import java.util.List;
import java.util.Objects;
import java.util.concurrent.ConcurrentHashMap;
Expand Down Expand Up @@ -442,4 +443,78 @@ public boolean includes(TopicName otherTopicName) {
public boolean isV2() {
return cluster == null;
}

/**
* 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("://");
Comment thread
BewareMyPower marked this conversation as resolved.
if (index >= 0) {
TopicDomain.getEnum(topic.substring(0, index));
final List<String> parts = splitBySlash(topic.substring(index + "://".length()), 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(TopicName.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,76 @@ public void topic() {
} catch (IllegalArgumentException e) {
// Ok
}
assertThrows(IllegalArgumentException.class, () -> TopicName.toFullTopicName("://tenant.namespace:my-topic"));

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

try {
TopicName.get("invalid://tenant/cluster/namespace/topic");
fail("Should have raied exception");
} catch (IllegalArgumentException e) {
// Ok
}
assertThrows(IllegalArgumentException.class,
() -> TopicName.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, () -> TopicName.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,
() -> TopicName.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,
() -> TopicName.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,
() -> TopicName.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,
() -> TopicName.toFullTopicName("persistent://pulsar/cluster/namespace/"));

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

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

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

try {
TopicName.get(" ");
fail("Should have raised exception");
} catch (IllegalArgumentException e) {
// Ok
}
assertThrows(IllegalArgumentException.class, () -> TopicName.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 +361,15 @@ 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=", TopicName.toFullTopicName("tp???xx="));
assertEquals("persistent://tenant/ns/tp???xx=", TopicName.toFullTopicName("tenant/ns/tp???xx="));
assertEquals("persistent://tenant/ns/test", TopicName.toFullTopicName("persistent://tenant/ns/test"));
assertThrows(IllegalArgumentException.class, () -> TopicName.toFullTopicName("ns/topic"));
// v1 format is not supported when the domain is not included
assertThrows(IllegalArgumentException.class, () -> TopicName.toFullTopicName("tenant/cluster/ns/topic"));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -221,7 +221,7 @@ public void handlePartitionMetadataResponse(CommandPartitionedTopicMetadata part
**/
private void handlePartitionMetadataResponse(CommandPartitionedTopicMetadata partitionMetadata,
long clientRequestId) {
TopicName topicName = TopicName.get(partitionMetadata.getTopic());
String topicName = TopicName.toFullTopicName(partitionMetadata.getTopic());

String serviceUrl = getBrokerServiceUrl(clientRequestId);
if (serviceUrl == null) {
Expand All @@ -235,7 +235,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 +245,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
Original file line number Diff line number Diff line change
Expand Up @@ -101,7 +101,8 @@ private void logging(Channel conn, BaseCommand.Type cmdtype, String info, List<R
private final BaseCommand cmd = new BaseCommand();

public void channelRead(ChannelHandlerContext ctx, Object msg) {
TopicName topicName;
String key;
String topicName;
List<RawMessage> messages = new ArrayList<>();
ByteBuf buffer = (ByteBuf) (msg);

Expand Down Expand Up @@ -130,16 +131,16 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) {
logging(ctx.channel(), cmd.getType(), "", null);
break;
}
topicName = TopicName.get(ParserProxyHandler.producerHashMap.get(cmd.getSend().getProducerId() + ","
+ ctx.channel().id()));
topicName = TopicName.toFullTopicName(ParserProxyHandler.producerHashMap.get(
cmd.getSend().getProducerId() + "," + ctx.channel().id()));
MutableLong msgBytes = new MutableLong(0);
MessageParser.parseMessage(topicName, -1L,
-1L, buffer, (message) -> {
messages.add(message);
msgBytes.add(message.getData().readableBytes());
}, maxMessageSize);
// update topic stats
TopicStats topicStats = this.service.getTopicStats().computeIfAbsent(topicName.toString(),
TopicStats topicStats = this.service.getTopicStats().computeIfAbsent(topicName,
topic -> new TopicStats());
topicStats.getMsgInRate().recordMultipleEvents(messages.size(), msgBytes.longValue());
logging(ctx.channel(), cmd.getType(), "", messages);
Expand All @@ -158,8 +159,9 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) {
logging(ctx.channel(), cmd.getType(), "", null);
break;
}
topicName = TopicName.get(ParserProxyHandler.consumerHashMap.get(cmd.getMessage().getConsumerId()
+ "," + peerChannelId));
topicName = TopicName.toFullTopicName(ParserProxyHandler.consumerHashMap.get(
cmd.getMessage().getConsumerId() + "," + peerChannelId));

msgBytes = new MutableLong(0);
MessageParser.parseMessage(topicName, -1L,
-1L, buffer, (message) -> {
Expand Down
Loading