Skip to content

Commit

Permalink
[fix] [client] PIP-344 Do not create partitioned metadata when callin…
Browse files Browse the repository at this point in the history
…g pulsarClient.getPartitionsForTopic(topicName) (apache#22206)

(cherry picked from commit 4e5c0bc)
(cherry picked from commit 9e59dd0)
  • Loading branch information
poorbarcode authored and nikhil-ctds committed Jun 24, 2024
1 parent 568be99 commit 4fab68d
Show file tree
Hide file tree
Showing 23 changed files with 687 additions and 88 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -557,19 +557,29 @@ protected CompletableFuture<PartitionedTopicMetadata> internalGetPartitionedMeta
boolean checkAllowAutoCreation) {
return getPartitionedTopicMetadataAsync(topicName, authoritative, checkAllowAutoCreation)
.thenCompose(metadata -> {
CompletableFuture<Void> ret;
if (metadata.partitions == 0 && !checkAllowAutoCreation) {
if (metadata.partitions > 1) {
// Some clients does not support partitioned topic.
return internalValidateClientVersionAsync().thenApply(__ -> metadata);
} else if (metadata.partitions == 1) {
return CompletableFuture.completedFuture(metadata);
} else {
// metadata.partitions == 0
// The topic may be a non-partitioned topic, so check if it exists here.
// However, when checkAllowAutoCreation is true, the client will create the topic if
// it doesn't exist. In this case, `partitions == 0` means the automatically created topic
// is a non-partitioned topic so we shouldn't check if the topic exists.
ret = internalCheckTopicExists(topicName);
} else if (metadata.partitions > 1) {
ret = internalValidateClientVersionAsync();
} else {
ret = CompletableFuture.completedFuture(null);
return pulsar().getBrokerService().isAllowAutoTopicCreationAsync(topicName)
.thenCompose(brokerAllowAutoTopicCreation -> {
if (checkAllowAutoCreation) {
// Whether it exists or not, auto create a non-partitioned topic by client.
return CompletableFuture.completedFuture(metadata);
} else {
// If it does not exist, response a Not Found error.
// Otherwise, response a non-partitioned metadata.
return internalCheckTopicExists(topicName).thenApply(__ -> metadata);
}
});
}
return ret.thenApply(__ -> metadata);
});
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -81,6 +81,8 @@
import org.apache.pulsar.broker.authentication.AuthenticationState;
import org.apache.pulsar.broker.intercept.BrokerInterceptor;
import org.apache.pulsar.broker.limiter.ConnectionController;
import org.apache.pulsar.broker.resources.NamespaceResources;
import org.apache.pulsar.broker.resources.TopicResources;
import org.apache.pulsar.broker.service.BrokerServiceException.ConsumerBusyException;
import org.apache.pulsar.broker.service.BrokerServiceException.ServerMetadataException;
import org.apache.pulsar.broker.service.BrokerServiceException.ServiceUnitNotReadyException;
Expand Down Expand Up @@ -578,35 +580,93 @@ protected void handlePartitionMetadataRequest(CommandPartitionedTopicMetadata pa
isTopicOperationAllowed(topicName, TopicOperation.LOOKUP, authenticationData, originalAuthData).thenApply(
isAuthorized -> {
if (isAuthorized) {
unsafeGetPartitionedTopicMetadataAsync(getBrokerService().pulsar(), topicName)
.handle((metadata, ex) -> {
if (ex == null) {
int partitions = metadata.partitions;
commandSender.sendPartitionMetadataResponse(partitions, requestId);
} else {
if (ex instanceof PulsarClientException) {
log.warn("Failed to authorize {} at [{}] on topic {} : {}", getRole(),
remoteAddress, topicName, ex.getMessage());
commandSender.sendPartitionMetadataResponse(ServerError.AuthorizationError,
ex.getMessage(), requestId);
// Get if exists, respond not found error if not exists.
getBrokerService().isAllowAutoTopicCreationAsync(topicName).thenAccept(brokerAllowAutoCreate -> {
boolean autoCreateIfNotExist = partitionMetadata.isMetadataAutoCreationEnabled();
if (!autoCreateIfNotExist) {
final NamespaceResources namespaceResources = getBrokerService().pulsar()
.getPulsarResources().getNamespaceResources();
final TopicResources topicResources = getBrokerService().pulsar().getPulsarResources()
.getTopicResources();
namespaceResources.getPartitionedTopicResources()
.getPartitionedTopicMetadataAsync(topicName, false)
.handle((metadata, getMetadataEx) -> {
if (getMetadataEx != null) {
log.error("{} {} Failed to get partition metadata", topicName,
ServerCnx.this.toString(), getMetadataEx);
writeAndFlush(
Commands.newPartitionMetadataResponse(ServerError.MetadataError,
"Failed to get partition metadata",
requestId));
} else if (metadata.isPresent()) {
commandSender.sendPartitionMetadataResponse(metadata.get().partitions,
requestId);
} else if (topicName.isPersistent()) {
topicResources.persistentTopicExists(topicName).thenAccept(exists -> {
if (exists) {
commandSender.sendPartitionMetadataResponse(0, requestId);
return;
}
writeAndFlush(Commands.newPartitionMetadataResponse(
ServerError.TopicNotFound, "", requestId));
}).exceptionally(ex -> {
log.error("{} {} Failed to get partition metadata", topicName,
ServerCnx.this.toString(), ex);
writeAndFlush(
Commands.newPartitionMetadataResponse(ServerError.MetadataError,
"Failed to check partition metadata",
requestId));
return null;
});
} else {
// Regarding non-persistent topic, we do not know whether it exists or not.
// Just return a non-partitioned metadata if partitioned metadata does not
// exist.
// Broker will respond a not found error when doing subscribing or producing if
// broker not allow to auto create topics.
commandSender.sendPartitionMetadataResponse(0, requestId);
}
return null;
}).whenComplete((ignore, ignoreEx) -> {
lookupSemaphore.release();
if (ignoreEx != null) {
log.error("{} {} Failed to handle partition metadata request", topicName,
ServerCnx.this.toString(), ignoreEx);
}
});
} else {
// Get if exists, create a new one if not exists.
unsafeGetPartitionedTopicMetadataAsync(getBrokerService().pulsar(), topicName)
.whenComplete((metadata, ex) -> {
lookupSemaphore.release();
if (ex == null) {
int partitions = metadata.partitions;
commandSender.sendPartitionMetadataResponse(partitions, requestId);
} else {
log.warn("Failed to get Partitioned Metadata [{}] {}: {}", remoteAddress,
topicName, ex.getMessage(), ex);
ServerError error = ServerError.ServiceNotReady;
if (ex instanceof RestException restException){
int responseCode = restException.getResponse().getStatus();
if (responseCode == NOT_FOUND.getStatusCode()){
error = ServerError.TopicNotFound;
} else if (responseCode < INTERNAL_SERVER_ERROR.getStatusCode()){
error = ServerError.MetadataError;
if (ex instanceof PulsarClientException) {
log.warn("Failed to authorize {} at [{}] on topic {} : {}", getRole(),
remoteAddress, topicName, ex.getMessage());
commandSender.sendPartitionMetadataResponse(ServerError.AuthorizationError,
ex.getMessage(), requestId);
} else {
log.warn("Failed to get Partitioned Metadata [{}] {}: {}", remoteAddress,
topicName, ex.getMessage(), ex);
ServerError error = ServerError.ServiceNotReady;
if (ex instanceof RestException restException){
int responseCode = restException.getResponse().getStatus();
if (responseCode == NOT_FOUND.getStatusCode()){
error = ServerError.TopicNotFound;
} else if (responseCode < INTERNAL_SERVER_ERROR.getStatusCode()){
error = ServerError.MetadataError;
}
}
commandSender.sendPartitionMetadataResponse(error, ex.getMessage(),
requestId);
}
commandSender.sendPartitionMetadataResponse(error, ex.getMessage(), requestId);
}
}
lookupSemaphore.release();
return null;
});
});
}
});
} else {
final String msg = "Client is not authorized to Get Partition Metadata";
log.warn("[{}] {} with role {} on topic {}", remoteAddress, msg, getPrincipal(), topicName);
Expand Down
Loading

0 comments on commit 4fab68d

Please sign in to comment.