Skip to content

Commit

Permalink
[improve][broker] Omit making a copy of CommandAck when there are no …
Browse files Browse the repository at this point in the history
…broker interceptors (apache#18997)

(cherry picked from commit 1154d0a)
  • Loading branch information
lhotari authored and michaeljmarshall committed Dec 21, 2022
1 parent af3e1c2 commit b75f068
Show file tree
Hide file tree
Showing 8 changed files with 50 additions and 79 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -777,8 +777,12 @@ public void start() throws PulsarServerException {
this.defaultOffloader = createManagedLedgerOffloader(defaultOffloadPolicies);

this.brokerInterceptor = BrokerInterceptors.load(config);
brokerService.setInterceptor(getBrokerInterceptor());
this.brokerInterceptor.initialize(this);
// use getter to support mocking getBrokerInterceptor method in tests
BrokerInterceptor interceptor = getBrokerInterceptor();
if (interceptor != null) {
brokerService.setInterceptor(interceptor);
interceptor.initialize(this);
}
brokerService.start();

// Load additional servlets
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -177,44 +177,6 @@ default void onFilter(ServletRequest request, ServletResponse response, FilterCh
*/
void initialize(PulsarService pulsarService) throws Exception;

BrokerInterceptor DISABLED = new BrokerInterceptorDisabled();

/**
* Broker interceptor disabled implementation.
*/
class BrokerInterceptorDisabled implements BrokerInterceptor {

@Override
public void onPulsarCommand(BaseCommand command, ServerCnx cnx) throws InterceptException {
// no-op
}

@Override
public void onConnectionClosed(ServerCnx cnx) {
// no-op
}

@Override
public void onWebserviceRequest(ServletRequest request) {
// no-op
}

@Override
public void onWebserviceResponse(ServletRequest request, ServletResponse response) {
// no-op
}

@Override
public void initialize(PulsarService pulsarService) throws Exception {
// no-op
}

@Override
public void close() {
// no-op
}
}

/**
* Close this broker interceptor.
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -89,7 +89,7 @@ public static BrokerInterceptor load(ServiceConfiguration conf) throws IOExcepti
if (interceptors != null && !interceptors.isEmpty()) {
return new BrokerInterceptors(interceptors);
} else {
return DISABLED;
return null;
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -38,6 +38,7 @@
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicLongFieldUpdater;
import org.apache.pulsar.broker.ServiceConfiguration;
import org.apache.pulsar.broker.intercept.BrokerInterceptor;
import org.apache.pulsar.broker.service.BrokerServiceException.TopicClosedException;
import org.apache.pulsar.broker.service.BrokerServiceException.TopicTerminatedException;
import org.apache.pulsar.broker.service.Topic.PublishContext;
Expand Down Expand Up @@ -68,6 +69,7 @@ public class Producer {
private final boolean userProvidedProducerName;
private final long producerId;
private final String appId;
private final BrokerInterceptor brokerInterceptor;
private Rate msgIn;
private Rate chunkedMessageRate;
// it records msg-drop rate only for non-persistent topic
Expand Down Expand Up @@ -151,6 +153,7 @@ public Producer(Topic topic, TransportCnx cnx, long producerId, String producerN
this.topicEpoch = topicEpoch;

this.clientAddress = cnx.clientSourceAddress();
this.brokerInterceptor = cnx.getBrokerService().getInterceptor();
}

/**
Expand Down Expand Up @@ -483,8 +486,8 @@ public void run() {
producer.chunkedMessageRate.recordEvent();
}
producer.publishOperationCompleted();
if (producer.cnx.getBrokerService().getInterceptor() != null){
producer.cnx.getBrokerService().getInterceptor().messageProduced(
if (producer.brokerInterceptor != null) {
producer.brokerInterceptor.messageProduced(
(ServerCnx) producer.cnx, producer, startTimeNs, ledgerId, entryId, this);
}
recycle();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -382,10 +382,12 @@ private void interceptAndWriteCommand(BaseCommand command) {
}

private void safeIntercept(BaseCommand command, ServerCnx cnx) {
try {
this.interceptor.onPulsarCommand(command, cnx);
} catch (Exception e) {
log.error("Failed to execute command {} on broker interceptor.", command.getType(), e);
if (this.interceptor != null) {
try {
this.interceptor.onPulsarCommand(command, cnx);
} catch (Exception e) {
log.error("Failed to execute command {} on broker interceptor.", command.getType(), e);
}
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -55,7 +55,6 @@
import java.util.stream.Collectors;
import javax.naming.AuthenticationException;
import javax.net.ssl.SSLSession;
import lombok.val;
import org.apache.bookkeeper.mledger.AsyncCallbacks;
import org.apache.bookkeeper.mledger.Entry;
import org.apache.bookkeeper.mledger.ManagedLedgerException;
Expand Down Expand Up @@ -170,6 +169,7 @@ public class ServerCnx extends PulsarHandler implements TransportCnx {
private final boolean enableSubscriptionPatternEvaluation;
private final int maxSubscriptionPatternLength;
private final TopicListService topicListService;
private final BrokerInterceptor brokerInterceptor;
private State state;
private volatile boolean isActive = true;
private String authRole = null;
Expand Down Expand Up @@ -280,6 +280,7 @@ public ServerCnx(PulsarService pulsar, String listenerName) {
this.maxSubscriptionPatternLength = conf.getSubscriptionPatternMaxLength();
this.topicListService = new TopicListService(pulsar, this,
enableSubscriptionPatternEvaluation, maxSubscriptionPatternLength);
this.brokerInterceptor = this.service != null ? this.service.getInterceptor() : null;
}

@Override
Expand All @@ -296,7 +297,7 @@ public void channelActive(ChannelHandlerContext ctx) throws Exception {
}
log.info("New connection from {}", remoteAddress);
this.ctx = ctx;
this.commandSender = new PulsarCommandSenderImpl(getBrokerService().getInterceptor(), this);
this.commandSender = new PulsarCommandSenderImpl(brokerInterceptor, this);
this.service.getPulsarStats().recordConnectionCreate();
cnxsPerThread.get().add(this);
}
Expand All @@ -307,7 +308,6 @@ public void channelInactive(ChannelHandlerContext ctx) throws Exception {
connectionController.decreaseConnection(ctx.channel().remoteAddress());
isActive = false;
log.info("Closed connection from {}", remoteAddress);
BrokerInterceptor brokerInterceptor = getBrokerService().getInterceptor();
if (brokerInterceptor != null) {
brokerInterceptor.onConnectionClosed(this);
}
Expand Down Expand Up @@ -670,7 +670,6 @@ private void completeConnect(int clientProtoVersion, String clientVersion, boole
if (isNotBlank(clientVersion) && !clientVersion.contains(" ") /* ignore default version: pulsar client */) {
this.clientVersion = clientVersion.intern();
}
BrokerInterceptor brokerInterceptor = getBrokerService().getInterceptor();
if (brokerInterceptor != null) {
brokerInterceptor.onConnectionCreated(this);
}
Expand Down Expand Up @@ -1130,8 +1129,8 @@ protected void handleSubscribe(final CommandSubscribe subscribe) {
log.info("[{}] Created subscription on topic {} / {}",
remoteAddress, topicName, subscriptionName);
commandSender.sendSuccessResponse(requestId);
if (getBrokerService().getInterceptor() != null){
getBrokerService().getInterceptor().consumerCreated(this, consumer, metadata);
if (brokerInterceptor != null) {
brokerInterceptor.consumerCreated(this, consumer, metadata);
}
} else {
// The consumer future was completed before by a close command
Expand Down Expand Up @@ -1475,9 +1474,9 @@ private void buildProducerAndAddTopic(Topic topic, long producerId, String produ
commandSender.sendProducerSuccessResponse(requestId, producerName,
producer.getLastSequenceId(), producer.getSchemaVersion(),
newTopicEpoch, true /* producer is ready now */);
if (getBrokerService().getInterceptor() != null) {
getBrokerService().getInterceptor().
producerCreated(this, producer, metadata);
if (brokerInterceptor != null) {
brokerInterceptor.
producerCreated(this, producer, metadata);
}
return;
} else {
Expand Down Expand Up @@ -1525,9 +1524,9 @@ private void buildProducerAndAddTopic(Topic topic, long producerId, String produ
commandSender.sendProducerSuccessResponse(requestId, producerName,
producer.getLastSequenceId(), producer.getSchemaVersion(),
Optional.empty(), false/* producer is not ready now */);
if (getBrokerService().getInterceptor() != null) {
getBrokerService().getInterceptor().
producerCreated(this, producer, metadata);
if (brokerInterceptor != null) {
brokerInterceptor.
producerCreated(this, producer, metadata);
}
}
});
Expand Down Expand Up @@ -1607,7 +1606,8 @@ protected void handleAck(CommandAck ack) {
final boolean hasRequestId = ack.hasRequestId();
final long requestId = hasRequestId ? ack.getRequestId() : 0;
final long consumerId = ack.getConsumerId();
final CommandAck finalAck = getBrokerService().getInterceptor() != null ? new CommandAck().copyFrom(ack) : null;
// It is necessary to make a copy of the CommandAck instance for the interceptor.
final CommandAck copyOfAckForInterceptor = brokerInterceptor != null ? new CommandAck().copyFrom(ack) : null;

if (consumerFuture != null && consumerFuture.isDone() && !consumerFuture.isCompletedExceptionally()) {
Consumer consumer = consumerFuture.getNow(null);
Expand All @@ -1616,8 +1616,8 @@ protected void handleAck(CommandAck ack) {
ctx.writeAndFlush(Commands.newAckResponse(
requestId, null, null, consumerId));
}
if (getBrokerService().getInterceptor() != null) {
getBrokerService().getInterceptor().messageAcked(this, consumer, finalAck);
if (brokerInterceptor != null) {
brokerInterceptor.messageAcked(this, consumer, copyOfAckForInterceptor);
}
}).exceptionally(e -> {
if (hasRequestId) {
Expand Down Expand Up @@ -2651,8 +2651,8 @@ public ChannelHandlerContext ctx() {

@Override
protected void interceptCommand(BaseCommand command) throws InterceptException {
if (getBrokerService().getInterceptor() != null) {
getBrokerService().getInterceptor().onPulsarCommand(command, this);
if (brokerInterceptor != null) {
brokerInterceptor.onPulsarCommand(command, this);
}
}

Expand Down Expand Up @@ -2935,17 +2935,15 @@ public ByteBufPair newMessageAndIntercept(long consumerId, long ledgerId, long e
ackSet, epoch);
ByteBufPair res = Commands.serializeCommandMessageWithSize(command, metadataAndPayload);
try {
val brokerInterceptor = getBrokerService().getInterceptor();
if (brokerInterceptor != null) {
brokerInterceptor.onPulsarCommand(command, this);

CompletableFuture<Consumer> consumerFuture = consumers.get(consumerId);
if (consumerFuture != null && consumerFuture.isDone() && !consumerFuture.isCompletedExceptionally()) {
Consumer consumer = consumerFuture.getNow(null);
}
CompletableFuture<Consumer> consumerFuture = consumers.get(consumerId);
if (consumerFuture != null && consumerFuture.isDone() && !consumerFuture.isCompletedExceptionally()) {
Consumer consumer = consumerFuture.getNow(null);
if (brokerInterceptor != null) {
brokerInterceptor.messageDispatched(this, consumer, ledgerId, entryId, metadataAndPayload);
}
} else {
log.debug("BrokerInterceptor is not set in newMessageAndIntercept");
}
} catch (Exception e) {
log.error("Exception occur when intercept messages.", e);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -66,7 +66,9 @@ public void doFilter(ServletRequest servletRequest, ServletResponse servletRespo
}
try {
RequestWrapper requestWrapper = new RequestWrapper((HttpServletRequest) servletRequest);
interceptor.onWebserviceRequest(requestWrapper);
if (interceptor != null) {
interceptor.onWebserviceRequest(requestWrapper);
}
filterChain.doFilter(requestWrapper, servletResponse);
} catch (InterceptException e) {
exceptionHandler.handle(servletResponse, e);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -99,20 +99,20 @@ public void setup() throws Exception {
doReturn(pulsarResources).when(pulsar).getPulsarResources();
});

eventLoopGroup = new NioEventLoopGroup();
brokerService = spyWithClassAndConstructorArgs(BrokerService.class, pulsar, eventLoopGroup);
PulsarServiceMockSupport.mockPulsarServiceProps(pulsar, () -> {
doReturn(brokerService).when(pulsar).getBrokerService();
});

serverCnx = spyWithClassAndConstructorArgs(ServerCnx.class, pulsar);
doReturn(true).when(serverCnx).isActive();
doReturn(true).when(serverCnx).isWritable();
doReturn(new InetSocketAddress("localhost", 1234)).when(serverCnx).clientAddress();
when(serverCnx.getRemoteEndpointProtocolVersion()).thenReturn(ProtocolVersion.v12.getValue());
when(serverCnx.ctx()).thenReturn(mock(ChannelHandlerContext.class));
doReturn(new PulsarCommandSenderImpl(null, serverCnx))
.when(serverCnx).getCommandSender();

eventLoopGroup = new NioEventLoopGroup();
brokerService = spyWithClassAndConstructorArgs(BrokerService.class, pulsar, eventLoopGroup);
PulsarServiceMockSupport.mockPulsarServiceProps(pulsar, () -> {
doReturn(brokerService).when(pulsar).getBrokerService();
});
.when(serverCnx).getCommandSender();

String topicName = TopicName.get("MessageCumulativeAckTest").toString();
PersistentTopic persistentTopic = new PersistentTopic(topicName, mock(ManagedLedger.class), brokerService);
Expand Down

0 comments on commit b75f068

Please sign in to comment.