Skip to content

Commit

Permalink
[pulsar-proxy] add rest-api to get connection and topic stats (apache…
Browse files Browse the repository at this point in the history
…#6473)

* [pulsar-proxy] add rest-api to get connection and topic stats

* fix rebasing

* fix stats url
  • Loading branch information
rdhabalia authored Mar 11, 2020
1 parent efe19e0 commit c802609
Show file tree
Hide file tree
Showing 22 changed files with 551 additions and 43 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,7 @@

import java.util.concurrent.TimeUnit;
import org.apache.bookkeeper.mledger.ManagedLedgerFactoryMXBean;
import org.apache.bookkeeper.mledger.util.Rate;
import org.apache.pulsar.common.stats.Rate;

@SuppressWarnings("checkstyle:javadoctype")
public class ManagedLedgerFactoryMBeanImpl implements ManagedLedgerFactoryMXBean {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,8 +23,8 @@
import org.apache.bookkeeper.mledger.ManagedCursor;
import org.apache.bookkeeper.mledger.ManagedLedgerMXBean;
import org.apache.bookkeeper.mledger.proto.PendingBookieOpsStats;
import org.apache.bookkeeper.mledger.util.Rate;
import org.apache.bookkeeper.mledger.util.StatsBuckets;
import org.apache.pulsar.common.stats.Rate;

public class ManagedLedgerMBeanImpl implements ManagedLedgerMXBean {

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -40,7 +40,6 @@
import org.apache.bookkeeper.mledger.Entry;
import org.apache.bookkeeper.mledger.Position;
import org.apache.bookkeeper.mledger.impl.PositionImpl;
import org.apache.bookkeeper.mledger.util.Rate;
import org.apache.bookkeeper.util.collections.ConcurrentLongLongPairHashMap;
import org.apache.bookkeeper.util.collections.ConcurrentLongLongPairHashMap.LongPair;
import org.apache.commons.lang3.mutable.MutableInt;
Expand All @@ -55,6 +54,7 @@
import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.policies.data.ConsumerStats;
import org.apache.pulsar.common.protocol.Commands;
import org.apache.pulsar.common.stats.Rate;
import org.apache.pulsar.common.util.DateFormatter;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -35,7 +35,6 @@
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicLongFieldUpdater;

import org.apache.bookkeeper.mledger.util.Rate;
import org.apache.pulsar.broker.authentication.AuthenticationDataSource;
import org.apache.pulsar.broker.service.BrokerServiceException.TopicClosedException;
import org.apache.pulsar.broker.service.BrokerServiceException.TopicTerminatedException;
Expand All @@ -49,6 +48,7 @@
import org.apache.pulsar.common.policies.data.NonPersistentPublisherStats;
import org.apache.pulsar.common.policies.data.PublisherStats;
import org.apache.pulsar.common.protocol.schema.SchemaVersion;
import org.apache.pulsar.common.stats.Rate;
import org.apache.pulsar.common.util.DateFormatter;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,11 +23,11 @@

import org.apache.bookkeeper.mledger.Entry;
import org.apache.bookkeeper.mledger.impl.PositionImpl;
import org.apache.bookkeeper.mledger.util.Rate;
import org.apache.pulsar.broker.service.BrokerServiceException;
import org.apache.pulsar.broker.service.Consumer;
import org.apache.pulsar.broker.service.Dispatcher;
import org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.SubType;
import org.apache.pulsar.common.stats.Rate;


public interface NonPersistentDispatcher extends Dispatcher {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,6 @@
import java.util.concurrent.atomic.AtomicIntegerFieldUpdater;

import org.apache.bookkeeper.mledger.Entry;
import org.apache.bookkeeper.mledger.util.Rate;
import org.apache.pulsar.broker.ServiceConfiguration;
import org.apache.pulsar.broker.service.AbstractDispatcherMultipleConsumers;
import org.apache.pulsar.broker.service.BrokerServiceException;
Expand All @@ -37,6 +36,7 @@
import org.apache.pulsar.broker.service.Subscription;
import org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.SubType;
import org.apache.pulsar.common.protocol.Commands;
import org.apache.pulsar.common.stats.Rate;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,6 @@
import java.util.List;

import org.apache.bookkeeper.mledger.Entry;
import org.apache.bookkeeper.mledger.util.Rate;
import org.apache.pulsar.broker.ServiceConfiguration;
import org.apache.pulsar.broker.admin.AdminResource;
import org.apache.pulsar.broker.service.AbstractDispatcherSingleActiveConsumer;
Expand All @@ -34,6 +33,7 @@
import org.apache.pulsar.broker.service.SendMessageInfo;
import org.apache.pulsar.broker.service.Subscription;
import org.apache.pulsar.common.protocol.Commands;
import org.apache.pulsar.common.stats.Rate;
import org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.SubType;
import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.policies.data.Policies;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,6 @@

import org.apache.bookkeeper.mledger.Entry;
import org.apache.bookkeeper.mledger.Position;
import org.apache.bookkeeper.mledger.util.Rate;
import org.apache.pulsar.broker.service.AbstractReplicator;
import org.apache.pulsar.broker.service.BrokerService;
import org.apache.pulsar.broker.service.BrokerServiceException.NamingException;
Expand All @@ -35,6 +34,7 @@
import org.apache.pulsar.client.impl.ProducerImpl;
import org.apache.pulsar.client.impl.SendCallback;
import org.apache.pulsar.common.policies.data.NonPersistentReplicatorStats;
import org.apache.pulsar.common.stats.Rate;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,8 +27,8 @@
import org.apache.bookkeeper.mledger.ManagedCursor;
import org.apache.bookkeeper.mledger.ManagedLedgerException;
import org.apache.bookkeeper.mledger.Position;
import org.apache.bookkeeper.mledger.util.Rate;
import org.apache.pulsar.client.impl.MessageImpl;
import org.apache.pulsar.common.stats.Rate;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -45,7 +45,6 @@
import org.apache.bookkeeper.mledger.ManagedLedgerException.CursorAlreadyClosedException;
import org.apache.bookkeeper.mledger.ManagedLedgerException.TooManyRequestsException;
import org.apache.bookkeeper.mledger.Position;
import org.apache.bookkeeper.mledger.util.Rate;
import org.apache.pulsar.broker.service.AbstractReplicator;
import org.apache.pulsar.broker.service.BrokerService;
import org.apache.pulsar.broker.service.BrokerServiceException.NamingException;
Expand All @@ -63,6 +62,7 @@
import org.apache.pulsar.common.policies.data.Policies;
import org.apache.pulsar.common.api.proto.PulsarMarkers.MarkerType;
import org.apache.pulsar.common.policies.data.ReplicatorStats;
import org.apache.pulsar.common.stats.Rate;
import org.apache.pulsar.common.util.Codec;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.bookkeeper.mledger.util;
package org.apache.pulsar.common.stats;

import static com.google.common.base.Preconditions.checkArgument;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,7 @@
import io.netty.handler.ssl.SslHandler;
import io.netty.util.concurrent.Future;
import io.netty.util.concurrent.FutureListener;
import lombok.Getter;

import java.net.URI;
import java.net.URISyntaxException;
Expand All @@ -53,15 +54,20 @@
import org.apache.pulsar.common.api.AuthData;
import org.apache.pulsar.common.protocol.Commands;
import org.apache.pulsar.common.protocol.PulsarDecoder;
import org.apache.pulsar.common.stats.Rate;
import org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthChallenge;
import org.apache.pulsar.common.api.proto.PulsarApi.CommandConnected;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

public class DirectProxyHandler {

@Getter
private Channel inboundChannel;
@Getter
Channel outboundChannel;
@Getter
private final Rate inboundChannelRequestsRate;
protected static Map<ChannelId, ChannelId> inboundOutboundChannelMap = new ConcurrentHashMap<>();
private String originalPrincipal;
private AuthData clientAuthData;
Expand All @@ -72,11 +78,14 @@ public class DirectProxyHandler {
private final Authentication authentication;
private final SslContext sslCtx;
private AuthenticationDataProvider authenticationDataProvider;
private ProxyService service;

public DirectProxyHandler(ProxyService service, ProxyConnection proxyConnection, String targetBrokerUrl,
int protocolVersion, SslContext sslCtx) {
this.service = service;
this.authentication = proxyConnection.getClientAuthentication();
this.inboundChannel = proxyConnection.ctx().channel();
this.inboundChannelRequestsRate = new Rate();
this.originalPrincipal = proxyConnection.clientAuthRole;
this.clientAuthData = proxyConnection.clientAuthData;
this.clientAuthMethod = proxyConnection.clientAuthMethod;
Expand Down Expand Up @@ -288,20 +297,20 @@ protected void handleConnected(CommandConnected connected) {
+ Commands.MESSAGE_SIZE_FRAME_PADDING, 0, 4, 0, 4));

inboundChannel.pipeline().addBefore("handler", "inboundParser",
new ParserProxyHandler(inboundChannel,
new ParserProxyHandler(service, inboundChannel,
ParserProxyHandler.FRONTEND_CONN,
connected.getMaxMessageSize()));
outboundChannel.pipeline().addBefore("proxyOutboundHandler", "outboundParser",
new ParserProxyHandler(outboundChannel,
new ParserProxyHandler(service, outboundChannel,
ParserProxyHandler.BACKEND_CONN,
connected.getMaxMessageSize()));
} else {
inboundChannel.pipeline().addBefore("handler", "inboundParser",
new ParserProxyHandler(inboundChannel,
new ParserProxyHandler(service, inboundChannel,
ParserProxyHandler.FRONTEND_CONN,
Commands.DEFAULT_MAX_MESSAGE_SIZE));
outboundChannel.pipeline().addBefore("proxyOutboundHandler", "outboundParser",
new ParserProxyHandler(outboundChannel,
new ParserProxyHandler(service, outboundChannel,
ParserProxyHandler.BACKEND_CONN,
Commands.DEFAULT_MAX_MESSAGE_SIZE));
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,24 +19,29 @@

package org.apache.pulsar.proxy.server;

import com.google.common.collect.Lists;
import io.netty.buffer.ByteBuf;
import io.netty.buffer.ByteBufUtil;
import io.netty.buffer.Unpooled;
import io.netty.buffer.CompositeByteBuf;
import io.netty.channel.Channel;
import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.ChannelInboundHandlerAdapter;
import java.util.List;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;

import org.apache.commons.lang3.mutable.MutableLong;
import org.apache.pulsar.common.api.proto.PulsarApi;
import org.apache.pulsar.common.api.raw.MessageParser;
import org.apache.pulsar.common.api.raw.RawMessage;
import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream;
import org.apache.pulsar.proxy.stats.TopicStats;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.List;
import java.util.concurrent.ConcurrentHashMap;
import java.util.Map;

import com.google.common.collect.Lists;

import io.netty.buffer.ByteBuf;
import io.netty.buffer.ByteBufUtil;
import io.netty.buffer.CompositeByteBuf;
import io.netty.buffer.Unpooled;
import io.netty.channel.Channel;
import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.ChannelInboundHandlerAdapter;


public class ParserProxyHandler extends ChannelInboundHandlerAdapter {
Expand All @@ -51,16 +56,18 @@ public class ParserProxyHandler extends ChannelInboundHandlerAdapter {
private String connType;

private int maxMessageSize;
private final ProxyService service;


//producerid+channelid as key
//or consumerid+channelid as key
private static Map<String, String> producerHashMap = new ConcurrentHashMap<>();
private static Map<String, String> consumerHashMap = new ConcurrentHashMap<>();

public ParserProxyHandler(Channel channel, String type, int maxMessageSize){
public ParserProxyHandler(ProxyService service, Channel channel, String type, int maxMessageSize) {
this.service = service;
this.channel = channel;
this.connType=type;
this.connType = type;
this.maxMessageSize = maxMessageSize;
}

Expand Down Expand Up @@ -117,11 +124,16 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) {
break;
}
topicName = TopicName.get(ParserProxyHandler.producerHashMap.get(String.valueOf(cmd.getSend().getProducerId()) + "," + String.valueOf(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(),
topic -> new TopicStats());
topicStats.getMsgInRate().recordMultipleEvents(messages.size(), msgBytes.longValue());
logging(ctx.channel() , cmd.getType() , "" , messages);
break;

Expand All @@ -137,12 +149,16 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) {
break;
}
topicName = TopicName.get(ParserProxyHandler.consumerHashMap.get(String.valueOf(cmd.getMessage().getConsumerId()) + "," + DirectProxyHandler.inboundOutboundChannelMap.get(ctx.channel().id())));
msgBytes = new MutableLong(0);
MessageParser.parseMessage(topicName, -1L,
-1L,buffer,(message) -> {
messages.add(message);
msgBytes.add(message.getData().readableBytes());
}, maxMessageSize);


// update topic stats
topicStats = this.service.getTopicStats().computeIfAbsent(topicName.toString(),
topic -> new TopicStats());
topicStats.getMsgOutRate().recordMultipleEvents(messages.size(), msgBytes.longValue());
logging(ctx.channel() , cmd.getType() , "" , messages);
break;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -59,6 +59,7 @@
import io.netty.handler.ssl.SslHandler;
import io.netty.util.concurrent.Future;
import io.netty.util.concurrent.FutureListener;
import lombok.Getter;

/**
* Handles incoming discovery request from client and sends appropriate response back to client
Expand All @@ -74,6 +75,7 @@ public class ProxyConnection extends PulsarHandler implements FutureListener<Voi
private final SslContext sslCtx;

private LookupProxyHandler lookupProxyHandler = null;
@Getter
private DirectProxyHandler directProxyHandler = null;
String clientAuthRole;
AuthData clientAuthData;
Expand Down Expand Up @@ -138,6 +140,7 @@ public void channelUnregistered(ChannelHandlerContext ctx) throws Exception {
public void channelActive(ChannelHandlerContext ctx) throws Exception {
super.channelActive(ctx);
ProxyService.newConnections.inc();
service.getClientCnxs().add(this);
LOG.info("[{}] New connection opened", remoteAddress);
}

Expand All @@ -152,7 +155,7 @@ public void channelInactive(ChannelHandlerContext ctx) throws Exception {
if (client != null) {
client.close();
}

service.getClientCnxs().remove(this);
LOG.info("[{}] Connection closed", remoteAddress);
}

Expand All @@ -179,7 +182,9 @@ public void channelRead(final ChannelHandlerContext ctx, Object msg) throws Exce
// only if we can write on the connection
ProxyService.opsCounter.inc();
if (msg instanceof ByteBuf) {
ProxyService.bytesCounter.inc(((ByteBuf) msg).readableBytes());
int bytes = ((ByteBuf) msg).readableBytes();
directProxyHandler.getInboundChannelRequestsRate().recordEvent(bytes);
ProxyService.bytesCounter.inc(bytes);
}
directProxyHandler.outboundChannel.writeAndFlush(msg).addListener(this);
break;
Expand Down
Loading

0 comments on commit c802609

Please sign in to comment.