diff --git a/.gitignore b/.gitignore
index f3fd3531cf321..e84c1b4c588f7 100644
--- a/.gitignore
+++ b/.gitignore
@@ -27,3 +27,6 @@ pulsar-broker/src/test/resources/log4j.properties
# Maven
log/
target/
+
+# tmp files
+*.pid
diff --git a/pom.xml b/pom.xml
index a2ece21af26ae..9b0e97363a053 100644
--- a/pom.xml
+++ b/pom.xml
@@ -484,6 +484,7 @@ flexible messaging model and an intuitive client API.
pulsar-client-cpp/lib/checksum/int_types.h
pulsar-client-cpp/lib/checksum/crc32c*
pulsar-client-cpp/lib/lz4/lz4.*
+ pulsar-client-cpp/.idea/*
pulsar-client-cpp/lib/PulsarApi.pb.*
pulsar-client-cpp/CMakeFiles/**
pulsar-client-cpp/travis-build.sh
diff --git a/protobuf/README.md b/protobuf/README.md
index ea03bbdc54cb6..11bb176df1343 100644
--- a/protobuf/README.md
+++ b/protobuf/README.md
@@ -3,31 +3,56 @@
Pulsar uses protocol buffer messages for the client/broker wire protocol.
-The protocol definition is located at `pulsar-common/src/main/proto/PulsarApi.proto` and the pre-generated Java code is at `pulsar-common/src/main/java/com/yahoo/pulsar/common/api/proto/PulsarApi.java`.
+The protocol definition is located at `pulsar-common/src/main/proto/PulsarApi.proto`. When making a change to the `PulsarApi.proto` definition, we have to regenerate the `PulsarApi.*` files and include them in the same commit.
-When making a change to the `PulsarApi.proto` definition, we have regenerate the `PulsarApi.java` and include that in the same commit.
+### For Broker and Java Client:
+
+The pre-generated Java code is at `pulsar-common/src/main/java/com/yahoo/pulsar/common/api/proto/PulsarApi.java`.
We are currently using a modified version of the Google Protocol Buffer code generator, to generate code that can serialize/deserialize messages with no memory allocations (caching already instantiated objects) and also to be able to directly use Netty pooled ByteBuf with direct memory.
-To re-generate the `PulsarApi.java` code you need to apply a patch to the protobuf generator. Patch is found at `protobuf.patch`.
+To re-generate the `PulsarApi.java` code you need to apply a patch to the protobuf generator. Patch is found in `protobuf.patch`.
+
+### For C++ Client:
+
+The pre-generated C++ code is at `pulsar-client-cpp/lib/PulsarApi.pb.cc` and `pulsar-client-cpp/lib/PulsarApi.pb.h`.
+
+### Commands for creating the pre-generated files
```shell
+export PULSAR_HOME=
+
+cd $HOME
git clone https://github.com/google/protobuf.git
-cd protobuf
-git checkout v2.4.1
-# Apply patch
-patch -p1 < ../pulsar-path/protobuf/protobuf.patch
+### For C++ ###
+cd ${HOME}/protobuf
+git checkout v2.6.0
-# Compile protobuf
+### Compile protobuf
+autoreconf --install
./configure
make
-# This would leave the binary in src/protoc
+### Re-generate PulsarApi
+cd ${PULSAR_HOME}/pulsar-client-cpp/
+export PROTOC=${HOME}/protobuf/src/protoc
+./generate_protobuf.sh
+
+### For Java ###
+cd ${HOME}/protobuf
+git checkout v2.4.1
+### Apply patch
+patch -p1 < ${PULSAR_HOME}/protobuf/protobuf.patch
-# Re-generate PulsarApi
-cd pulsar-path/pulsar-common
-PROTOC=~/protobuf/src/protoc ./generate_protobuf.sh
+### Compile protobuf
+autoreconf --install
+./configure
+make
+### Re-generate PulsarApi
+cd ${PULSAR_HOME}/pulsar-common/
+export PROTOC=${HOME}/protobuf/src/protoc
+./generate_protobuf.sh
```
diff --git a/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/service/Dispatcher.java b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/service/Dispatcher.java
index 4d6f7c0abca0a..426ba08326a1d 100644
--- a/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/service/Dispatcher.java
+++ b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/service/Dispatcher.java
@@ -18,7 +18,6 @@
import java.util.List;
import java.util.concurrent.CompletableFuture;
-import com.yahoo.pulsar.common.api.proto.PulsarApi;
import com.yahoo.pulsar.common.api.proto.PulsarApi.CommandSubscribe.SubType;
import com.yahoo.pulsar.utils.CopyOnWriteArrayList;
import org.apache.bookkeeper.mledger.impl.PositionImpl;
diff --git a/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/service/ServerCnx.java b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/service/ServerCnx.java
index e927226e2604f..ee8a66fbac439 100644
--- a/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/service/ServerCnx.java
+++ b/pulsar-broker/src/main/java/com/yahoo/pulsar/broker/service/ServerCnx.java
@@ -33,13 +33,18 @@
import com.yahoo.pulsar.broker.authentication.AuthenticationDataCommand;
import com.yahoo.pulsar.broker.service.BrokerServiceException.ServiceUnitNotReadyException;
+import com.yahoo.pulsar.broker.service.persistent.PersistentSubscription;
+import com.yahoo.pulsar.broker.service.persistent.PersistentTopic;
import com.yahoo.pulsar.client.api.PulsarClientException;
+import com.yahoo.pulsar.common.policies.data.ConsumerStats;
import com.yahoo.pulsar.common.api.Commands;
import com.yahoo.pulsar.common.api.PulsarHandler;
import com.yahoo.pulsar.common.api.proto.PulsarApi.CommandAck;
import com.yahoo.pulsar.common.api.proto.PulsarApi.CommandCloseConsumer;
import com.yahoo.pulsar.common.api.proto.PulsarApi.CommandCloseProducer;
import com.yahoo.pulsar.common.api.proto.PulsarApi.CommandConnect;
+import com.yahoo.pulsar.common.api.proto.PulsarApi.CommandConsumerStats;
+import com.yahoo.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse;
import com.yahoo.pulsar.common.api.proto.PulsarApi.CommandFlow;
import com.yahoo.pulsar.common.api.proto.PulsarApi.CommandLookupTopic;
import com.yahoo.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadata;
@@ -192,6 +197,88 @@ protected void handlePartitionMetadataRequest(CommandPartitionedTopicMetadata pa
});
}
+ @Override
+ protected void handleConsumerStats(CommandConsumerStats commandConsumerStats) {
+ if (log.isDebugEnabled()) {
+ log.debug("Received CommandConsumerStats call from {}", remoteAddress);
+ }
+
+ final long requestId = commandConsumerStats.getRequestId();
+ final String topicName = commandConsumerStats.getTopicName();
+ final String subscriptionName = commandConsumerStats.getSubscriptionName();
+ final long consumerId = commandConsumerStats.getConsumerId();
+
+ if (log.isDebugEnabled()) {
+ log.debug("CommandConsumerStats[requestId = {}, topicName = {}, subscriptionName = {}, consumerId = {}]", requestId, topicName, subscriptionName, consumerId);
+ }
+
+ ByteBuf msg = null;
+ try {
+ PersistentTopic topic = (PersistentTopic) getBrokerService().getTopicReference(topicName);
+ if (topic != null) {
+ if (topic.getSubscriptions().containsKey(subscriptionName)) {
+ PersistentSubscription subscription = topic.getSubscriptions().get(subscriptionName);
+ boolean consumerFound = false;
+ for (Consumer consumer : subscription.getConsumers()) {
+ if (consumer.consumerId() == consumerId) {
+ consumerFound = true;
+ msg = Commands.newConsumerStatsResponse(createConsumerStatsResponse(consumer, subscription, requestId));
+ break;
+ }
+ }
+ if (!consumerFound) {
+ log.error(
+ "Failed to get consumer-stats response - Consumer not found for CommandConsumerStats[remoteAddress = {}, requestId = {}, topicName = {}, subscriptionName = {}, consumerId = {}]",
+ remoteAddress, requestId, topicName, subscriptionName, consumerId);
+ msg = Commands.newConsumerStatsResponse(ServerError.ConsumerNotFound,
+ "Consumer " + consumerId + " not found", requestId);
+ }
+ } else {
+ log.error(
+ "Failed to get consumer-stats response - Subscription not found for CommandConsumerStats[remoteAddress = {}, requestId = {}, topicName = {}, subscriptionName = {}, consumerId = {}]",
+ remoteAddress, requestId, topicName, subscriptionName, consumerId);
+ msg = Commands.newConsumerStatsResponse(ServerError.SubscriptionNotFound,
+ "Subscription " + subscriptionName + " not found", requestId);
+ }
+ } else {
+ log.error(
+ "Failed to get consumer-stats response - Topic not found for CommandConsumerStats[remoteAddress = {}, requestId = {}, topicName = {}, subscriptionName = {}, consumerId = {}]",
+ remoteAddress, requestId, topicName, subscriptionName, consumerId);
+ msg = Commands.newConsumerStatsResponse(ServerError.TopicNotFound, "Topic " + topicName + " not found",
+ requestId);
+ }
+ } catch (Exception e) {
+ log.error("Failed to get consumer-stats response - Exception: {} for CommandConsumerStats[remoteAddress = {}, requestId = {}, topicName = {}, subscriptionName = {}, consumerId = {}]",
+ e, remoteAddress, requestId, topicName, subscriptionName, consumerId);
+ msg = Commands.newConsumerStatsResponse(ServerError.UnknownError, "Exception: " + e, requestId);
+ } finally {
+ if (msg != null) {
+ ctx.writeAndFlush(msg);
+ }
+ }
+ }
+
+ CommandConsumerStatsResponse.Builder createConsumerStatsResponse(Consumer consumer, PersistentSubscription subscription, long requestId) {
+ CommandConsumerStatsResponse.Builder commandConsumerStatsResponseBuilder = CommandConsumerStatsResponse
+ .newBuilder();
+ ConsumerStats consumerStats = consumer.getStats();
+ commandConsumerStatsResponseBuilder.setRequestId(requestId);
+ commandConsumerStatsResponseBuilder.setMsgRateOut(consumerStats.msgRateOut);
+ commandConsumerStatsResponseBuilder.setMsgThroughputOut(consumerStats.msgThroughputOut);
+ commandConsumerStatsResponseBuilder.setMsgRateRedeliver(consumerStats.msgRateRedeliver);
+ commandConsumerStatsResponseBuilder.setConsumerName(consumerStats.consumerName);
+ commandConsumerStatsResponseBuilder.setAvailablePermits(consumerStats.availablePermits);
+ commandConsumerStatsResponseBuilder.setUnackedMessages(consumerStats.unackedMessages);
+ commandConsumerStatsResponseBuilder.setBlockedConsumerOnUnackedMsgs(consumerStats.blockedConsumerOnUnackedMsgs);
+ commandConsumerStatsResponseBuilder.setAddress(consumerStats.address);
+ commandConsumerStatsResponseBuilder.setConnectedSince(consumerStats.connectedSince);
+
+ commandConsumerStatsResponseBuilder.setType(subscription.getTypeString());
+ commandConsumerStatsResponseBuilder.setMsgRateExpired(subscription.getExpiredMessageRate());
+ commandConsumerStatsResponseBuilder.setMsgBacklog(subscription.getNumberOfEntriesInBacklog());
+ return commandConsumerStatsResponseBuilder;
+ }
+
@Override
protected void handleConnect(CommandConnect connect) {
checkArgument(state == State.Start);
diff --git a/pulsar-client-cpp/.gitignore b/pulsar-client-cpp/.gitignore
index b64d5e7450c0f..86cfc587758c6 100644
--- a/pulsar-client-cpp/.gitignore
+++ b/pulsar-client-cpp/.gitignore
@@ -48,6 +48,7 @@
.settings/
.pydevproject
.idea/
+*.cbp
# doxygen files
apidocs/
diff --git a/pulsar-client-cpp/include/pulsar/Consumer.h b/pulsar-client-cpp/include/pulsar/Consumer.h
index e7621fd9a7d32..598c6b1319a63 100644
--- a/pulsar-client-cpp/include/pulsar/Consumer.h
+++ b/pulsar-client-cpp/include/pulsar/Consumer.h
@@ -21,7 +21,8 @@
#include
#include
#include
-
+#include
+#include
#pragma GCC visibility push(default)
class PulsarFriend;
@@ -55,6 +56,61 @@ enum ConsumerType {
ConsumerFailover
};
+class BrokerConsumerStats {
+ private:
+ /*
+ * validTillInMs_ - Stats will be valid till this time.
+ */
+ boost::posix_time::ptime validTill_;
+ public:
+ BrokerConsumerStats();
+ BrokerConsumerStats(boost::posix_time::ptime& validTill, double msgRateOut, double msgThroughputOut,
+ double msgRateRedeliver, std::string consumerName, int availablePermits,
+ int unackedMessages, bool blockedConsumerOnUnackedMsgs, std::string address,
+ std::string connectedSince, std::string type, double msgRateExpired, long msgBacklog);
+
+ /** Returns true if the Message is Expired **/
+ bool isValid() const;
+
+ /** Total rate of messages delivered to the consumer. msg/s */
+ double msgRateOut_;
+
+ /** Total throughput delivered to the consumer. bytes/s */
+ double msgThroughputOut_;
+
+ /** Total rate of messages redelivered by this consumer. msg/s */
+ double msgRateRedeliver_;
+
+ /** Name of the consumer */
+ std::string consumerName_;
+
+ /** Number of available message permits for the consumer */
+ int availablePermits_;
+
+ /** Number of unacknowledged messages for the consumer */
+ int unackedMessages_;
+
+ /** Flag to verify if consumer is blocked due to reaching threshold of unacked messages */
+ bool blockedConsumerOnUnackedMsgs_;
+
+ /** Address of this consumer */
+ std::string address_;
+
+ /** Timestamp of connection */
+ std::string connectedSince_;
+
+ /// Whether this subscription is Exclusive or Shared or Failover
+ std::string type_;
+
+ /// Total rate of messages expired on this subscription. msg/s
+ double msgRateExpired_;
+
+ /// Number of messages in the subscription backlog
+ long msgBacklog_;
+
+ friend std::ostream& operator<<(std::ostream& os, const BrokerConsumerStats& obj);
+};
+
/**
* Class specifying the configuration of a consumer.
*/
@@ -284,7 +340,20 @@ class Consumer {
*/
void redeliverUnacknowledgedMessages();
- private:
+ /**
+ * Gets Consumer Stats from broker.
+ * The stats are cached for 30 seconds, if a call is made before the stats returned by the previous call expires
+ * then cached data will be returned. BrokerConsumerStats::isValid() function can be used to check if the stats are
+ * still valid.
+ *
+ * @param brokerConsumerStats - if the function returns ResultOk, this object will contain consumer stats
+ * @param partitionIndex - optional parameter which is to be populated only if the topic is partitioned.
+ *
+ * @note This is a blocking call with timeout of thirty seconds.
+ */
+ Result getConsumerStats(BrokerConsumerStats& brokerConsumerStats, int partitionIndex = -1);
+
+private:
typedef boost::shared_ptr ConsumerImplBasePtr;
friend class PulsarFriend;
ConsumerImplBasePtr impl_;
diff --git a/pulsar-client-cpp/include/pulsar/Result.h b/pulsar-client-cpp/include/pulsar/Result.h
index 0619ed746e47b..ef5f02ca5ffe0 100644
--- a/pulsar-client-cpp/include/pulsar/Result.h
+++ b/pulsar-client-cpp/include/pulsar/Result.h
@@ -63,8 +63,11 @@ enum Result {
ResultProducerBlockedQuotaExceededError, /// Producer is blocked
ResultProducerBlockedQuotaExceededException, /// Producer is getting exception
ResultProducerQueueIsFull, /// Producer queue is full
-
- ResultMessageTooBig /// Trying to send a messages exceeding the max size
+ ResultMessageTooBig, /// Trying to send a messages exceeding the max size
+ ResultTopicNotFound, /// Topic not found
+ ResultSubscriptionNotFound, /// Subscription not found
+ ResultConsumerNotFound, /// Consumer not found
+ ResultUnsupportedVersionError /// Error when an older client/version doesn't support a required feature
};
// Return string representation of result code
diff --git a/pulsar-client-cpp/lib/BatchMessageContainer.cc b/pulsar-client-cpp/lib/BatchMessageContainer.cc
index 7fee02a8189f3..9dd38eba12b15 100644
--- a/pulsar-client-cpp/lib/BatchMessageContainer.cc
+++ b/pulsar-client-cpp/lib/BatchMessageContainer.cc
@@ -117,7 +117,8 @@ SharedBuffer BatchMessageContainer::getBatchedPayload() {
void BatchMessageContainer::clear() {
LOG_DEBUG(*this << " BatchMessageContainer::clear() called");
timer_->cancel();
- averageBatchSize_ = (messagesContainerListPtr_->size() + (averageBatchSize_ * numberOfBatchesSent_))/++numberOfBatchesSent_;
+ averageBatchSize_ = (messagesContainerListPtr_->size() + (averageBatchSize_ * numberOfBatchesSent_))/(numberOfBatchesSent_ + 1);
+ numberOfBatchesSent_++;
messagesContainerListPtr_ = messageContainerListPool.create();
// Try to optimize this
messagesContainerListPtr_->reserve(10000);
diff --git a/pulsar-client-cpp/lib/Client.cc b/pulsar-client-cpp/lib/Client.cc
index b870fa10b1c1f..505f2b60d5604 100644
--- a/pulsar-client-cpp/lib/Client.cc
+++ b/pulsar-client-cpp/lib/Client.cc
@@ -45,7 +45,9 @@ struct ClientConfiguration::Impl {
operationTimeoutSeconds(30),
messageListenerThreads(1),
concurrentLookupRequest(5000),
- logConfFilePath() {}
+ logConfFilePath(),
+ useTls(false),
+ tlsAllowInsecureConnection(true) {}
};
ClientConfiguration::ClientConfiguration()
diff --git a/pulsar-client-cpp/lib/ClientConnection.cc b/pulsar-client-cpp/lib/ClientConnection.cc
index d88f8f3bd2d77..bcb35be7a64bd 100644
--- a/pulsar-client-cpp/lib/ClientConnection.cc
+++ b/pulsar-client-cpp/lib/ClientConnection.cc
@@ -83,6 +83,18 @@ static Result getResult(ServerError serverError) {
case ProducerBlockedQuotaExceededException:
return ResultProducerBlockedQuotaExceededException;
+
+ case TopicNotFound:
+ return ResultTopicNotFound;
+
+ case SubscriptionNotFound:
+ return ResultSubscriptionNotFound;
+
+ case ConsumerNotFound:
+ return ResultConsumerNotFound;
+
+ case UnsupportedVersionError:
+ return ResultUnsupportedVersionError;
}
// NOTE : Do not add default case in the switch above. In future if we get new cases for
// ServerError and miss them in the switch above we would like to get notified. Adding
@@ -111,6 +123,8 @@ outgoingCmd_(),
havePendingPingRequest_(false),
keepAliveTimer_(),
maxPendingLookupRequest_(clientConfiguration.getConcurrentLookupRequest()),
+consumerStatsRequestTimer_(executor_->createDeadlineTimer()),
+consumerStatsTTLMs_(30 * 1000),
numOfPendingLookupRequest_(0),
isTlsAllowInsecureConnection_(false) {
if (clientConfiguration.isUseTls()) {
@@ -184,6 +198,42 @@ void ClientConnection::handlePulsarConnected(const CommandConnected& cmdConnecte
keepAliveTimer_->async_wait(
boost::bind(&ClientConnection::handleKeepAliveTimeout, shared_from_this()));
}
+
+ if (serverProtocolVersion_ >= v7) {
+ startConsumerStatsTimer(std::vector());
+ }
+}
+
+void ClientConnection::startConsumerStatsTimer(std::vector consumerStatsRequests) {
+ std::vector > consumerStatsPromises;
+ Lock lock(mutex_);
+
+ for (int i = 0; i < consumerStatsRequests.size(); i++) {
+ PendingConsumerStatsMap::iterator it = pendingConsumerStatsMap_.find(consumerStatsRequests[i]);
+ if (it != pendingConsumerStatsMap_.end()) {
+ LOG_DEBUG(cnxString_ << " removing request_id " << it->first << " from the pendingConsumerStatsMap_");
+ consumerStatsPromises.push_back(it->second);
+ pendingConsumerStatsMap_.erase(it);
+ } else {
+ LOG_DEBUG(cnxString_ << "request_id " << it->first << " already fulfilled - not removing it");
+ }
+ }
+
+ consumerStatsRequests.clear();
+ for (PendingConsumerStatsMap::iterator it = pendingConsumerStatsMap_.begin();
+ it != pendingConsumerStatsMap_.end(); ++it) {
+ consumerStatsRequests.push_back(it->first);
+ }
+ consumerStatsRequestTimer_->expires_from_now(operationsTimeout_);
+ consumerStatsRequestTimer_->async_wait(
+ boost::bind(&ClientConnection::handleConsumerStatsTimeout, shared_from_this(),
+ boost::asio::placeholders::error, consumerStatsRequests));
+ lock.unlock();
+ // Complex logic since promises need to be fulfilled outside the lock
+ for (int i = 0; i < consumerStatsPromises.size(); i++) {
+ consumerStatsPromises[i].setFailed(ResultTimeout);
+ LOG_WARN(cnxString_ << " Operation timedout, didn't get response from broker");
+ }
}
/// The number of unacknowledged probes to send before considering the connection dead and notifying the application layer
@@ -655,6 +705,57 @@ void ClientConnection::handleIncomingCommand() {
break;
}
+ case BaseCommand::CONSUMER_STATS_RESPONSE: {
+ const CommandConsumerStatsResponse& consumerStatsResponse = incomingCmd_.consumerstatsresponse();
+ LOG_DEBUG(
+ cnxString_
+ << "ConsumerStatsResponse command - Received consumer stats response from server. req_id: "
+ << consumerStatsResponse.request_id());
+ Lock lock(mutex_);
+ PendingConsumerStatsMap::iterator it = pendingConsumerStatsMap_.find(
+ consumerStatsResponse.request_id());
+ if (it != pendingConsumerStatsMap_.end()) {
+ Promise consumerStatsPromise = it->second;
+ pendingConsumerStatsMap_.erase(it);
+ lock.unlock();
+
+ if (consumerStatsResponse.has_error_code()) {
+ if (consumerStatsResponse.has_error_message()) {
+ LOG_ERROR(cnxString_ << " Failed to get consumer stats - "
+ << consumerStatsResponse.error_message());
+ }
+ consumerStatsPromise.setFailed(getResult(consumerStatsResponse.error_code()));
+ } else {
+ LOG_DEBUG(
+ cnxString_
+ << "ConsumerStatsResponse command - Received consumer stats response from server. req_id: "
+ << consumerStatsResponse.request_id() << " Stats: ");
+ boost::posix_time::ptime validTill = now() + milliseconds(consumerStatsTTLMs_);
+ BrokerConsumerStats brokerStats =
+ BrokerConsumerStats(validTill,
+ consumerStatsResponse.msgrateout(),
+ consumerStatsResponse.msgthroughputout(),
+ consumerStatsResponse.msgrateredeliver(),
+ consumerStatsResponse.consumername(),
+ consumerStatsResponse.availablepermits(),
+ consumerStatsResponse.unackedmessages(),
+ consumerStatsResponse.blockedconsumeronunackedmsgs(),
+ consumerStatsResponse.address(),
+ consumerStatsResponse.connectedsince(),
+ consumerStatsResponse.type(),
+ consumerStatsResponse.msgrateexpired(),
+ consumerStatsResponse.msgbacklog());
+ consumerStatsPromise.setValue(brokerStats);
+ }
+ } else {
+ LOG_WARN(
+ "ConsumerStatsResponse command - Received unknown request id from server: "
+ << consumerStatsResponse.request_id());
+ }
+ break;
+
+ }
+
case BaseCommand::LOOKUP_RESPONSE: {
const CommandLookupTopicResponse& lookupTopicResponse = incomingCmd_.lookuptopicresponse();
LOG_DEBUG(
@@ -822,6 +923,22 @@ void ClientConnection::handleIncomingCommand() {
}
}
+Future
+ClientConnection::newConsumerStats(const std::string topicName, const std::string subscriptionName,
+ uint64_t consumerId, uint64_t requestId) {
+ Lock lock(mutex_);
+ Promise promise;
+ if (isClosed()) {
+ lock.unlock();
+ LOG_ERROR(cnxString_ << " Client is not connected to the broker");
+ promise.setFailed(ResultNotConnected);
+ }
+ pendingConsumerStatsMap_.insert(std::make_pair(requestId, promise));
+ lock.unlock();
+ sendCommand(Commands::newConsumerStats(outgoingCmd_, topicName, subscriptionName, consumerId, requestId));
+ return promise.getFuture();
+}
+
void ClientConnection::newTopicLookup(const std::string& destinationName, bool authoritative,
const uint64_t requestId,
LookupDataResultPromisePtr promise) {
@@ -996,6 +1113,15 @@ void ClientConnection::handleKeepAliveTimeout() {
}
}
+void ClientConnection::handleConsumerStatsTimeout(const boost::system::error_code& ec,
+ std::vector consumerStatsRequests) {
+ if (ec) {
+ LOG_DEBUG(cnxString_ << " Ignoring timer cancelled event, code[" << ec << "]");
+ return;
+ }
+ startConsumerStatsTimer(consumerStatsRequests);
+}
+
void ClientConnection::close() {
Lock lock(mutex_);
state_ = Disconnected;
@@ -1009,28 +1135,45 @@ void ClientConnection::close() {
keepAliveTimer_->cancel();
}
- for (ProducersMap::iterator it = producers_.begin(); it != producers_.end(); ++it ) {
- HandlerBase::handleDisconnection(ResultConnectError, shared_from_this(), it->second);
+ if (consumerStatsRequestTimer_) {
+ consumerStatsRequestTimer_->cancel();
+ }
+ for (ProducersMap::iterator it = producers_.begin(); it != producers_.end(); ++it) {
+ HandlerBase::handleDisconnection(ResultConnectError, shared_from_this(), it->second);
}
- for (ConsumersMap::iterator it = consumers_.begin(); it != consumers_.end(); ++it ) {
+ for (ConsumersMap::iterator it = consumers_.begin(); it != consumers_.end(); ++it) {
HandlerBase::handleDisconnection(ResultConnectError, shared_from_this(), it->second);
}
connectPromise_.setFailed(ResultConnectError);
// Fail all pending operations on the connection
- for (PendingRequestsMap::iterator it = pendingRequests_.begin(); it != pendingRequests_.end(); ++it) {
+ for (PendingRequestsMap::iterator it = pendingRequests_.begin(); it != pendingRequests_.end(); ++it ) {
it->second.promise.setFailed(ResultConnectError);
}
// Fail all pending lookup-requests on the connection
lock.lock();
- for (PendingLookupRequestsMap::iterator it = pendingLookupRequests_.begin(); it != pendingLookupRequests_.end(); ++it) {
+ PendingLookupRequestsMap pendingLookupRequests;
+ pendingLookupRequests_.swap(pendingLookupRequests);
+ numOfPendingLookupRequest_ -= pendingLookupRequests.size();
+
+ PendingConsumerStatsMap pendingConsumerStatsMap;
+ pendingConsumerStatsMap_.swap(pendingConsumerStatsMap);
+ lock.unlock();
+
+ for (PendingLookupRequestsMap::iterator it = pendingLookupRequests.begin(); it != pendingLookupRequests.end(); ++it) {
it->second->setFailed(ResultConnectError);
- numOfPendingLookupRequest_--;
}
- lock.unlock();
+
+
+ for (PendingConsumerStatsMap::iterator it = pendingConsumerStatsMap.begin();
+ it != pendingConsumerStatsMap.end(); ++it) {
+ LOG_ERROR(cnxString_ << " Closing Client Connection, please try again later");
+ it->second.setFailed(ResultConnectError);
+ }
+
if (tlsSocket_) {
tlsSocket_->lowest_layer().close();
}
@@ -1080,5 +1223,4 @@ Commands::ChecksumType ClientConnection::getChecksumType() const {
return getServerProtocolVersion() >= proto::v6 ?
Commands::Crc32c : Commands::None;
}
-
}
diff --git a/pulsar-client-cpp/lib/ClientConnection.h b/pulsar-client-cpp/lib/ClientConnection.h
index 25123cdc82ab0..0617b46b58aa3 100644
--- a/pulsar-client-cpp/lib/ClientConnection.h
+++ b/pulsar-client-cpp/lib/ClientConnection.h
@@ -39,6 +39,7 @@
#include "LookupDataResult.h"
#include "UtilAllocator.h"
#include
+#include
using namespace pulsar;
@@ -131,7 +132,10 @@ class ClientConnection : public boost::enable_shared_from_this
Commands::ChecksumType getChecksumType() const;
+ Future newConsumerStats(const std::string topicName, const std::string subscriptionName,
+ uint64_t consumerId, uint64_t requestId) ;
private:
+ long consumerStatsTTLMs_ ;
struct PendingRequestData {
Promise promise;
@@ -250,6 +254,10 @@ class ClientConnection : public boost::enable_shared_from_this
typedef std::map ConsumersMap;
ConsumersMap consumers_;
+ typedef std::map > PendingConsumerStatsMap;
+ PendingConsumerStatsMap pendingConsumerStatsMap_;
+
+
boost::mutex mutex_;
typedef boost::unique_lock Lock;
@@ -266,10 +274,14 @@ class ClientConnection : public boost::enable_shared_from_this
// Signals whether we're waiting for a response from broker
bool havePendingPingRequest_;
DeadlineTimerPtr keepAliveTimer_;
+ DeadlineTimerPtr consumerStatsRequestTimer_;
+ void handleConsumerStatsTimeout(const boost::system::error_code &ec,
+ std::vector consumerStatsRequests);
+
+ void startConsumerStatsTimer(std::vector consumerStatsRequests);
uint32_t maxPendingLookupRequest_;
uint32_t numOfPendingLookupRequest_;
-
friend class PulsarFriend;
bool isTlsAllowInsecureConnection_;
diff --git a/pulsar-client-cpp/lib/Commands.cc b/pulsar-client-cpp/lib/Commands.cc
index d10dbd2785b54..75dae86a3879f 100644
--- a/pulsar-client-cpp/lib/Commands.cc
+++ b/pulsar-client-cpp/lib/Commands.cc
@@ -59,6 +59,17 @@ SharedBuffer Commands::newLookup(BaseCommand& cmd, const std::string& topic, con
return writeMessageWithSize(cmd);
}
+SharedBuffer Commands::newConsumerStats(BaseCommand& cmd, const std::string& topicName, const std::string& subscriptionName,
+ uint64_t consumerId, uint64_t requestId) {
+ cmd.set_type(BaseCommand::CONSUMER_STATS);
+ CommandConsumerStats* consumerStats = cmd.mutable_consumerstats();
+ consumerStats->set_topic_name(topicName);
+ consumerStats->set_subscription_name(subscriptionName);
+ consumerStats->set_consumer_id(consumerId);
+ consumerStats->set_request_id(requestId);
+ return writeMessageWithSize(cmd);
+}
+
PairSharedBuffer Commands::newSend(SharedBuffer& headers, BaseCommand& cmd,
uint64_t producerId, uint64_t sequenceId, ChecksumType checksumType, const Message& msg) {
const proto::MessageMetadata& metadata = msg.impl_->metadata;
@@ -318,6 +329,12 @@ std::string Commands::messageType(BaseCommand_Type type) {
case BaseCommand::LOOKUP_RESPONSE:
return "LOOKUP_RESPONSE";
break;
+ case BaseCommand::CONSUMER_STATS:
+ return "CONSUMER_STATS";
+ break;
+ case BaseCommand::CONSUMER_STATS_RESPONSE:
+ return "CONSUMER_STATS_RESPONSE";
+ break;
};
}
diff --git a/pulsar-client-cpp/lib/Commands.h b/pulsar-client-cpp/lib/Commands.h
index bab7f59b3bfac..0d4eeb4a3dbba 100644
--- a/pulsar-client-cpp/lib/Commands.h
+++ b/pulsar-client-cpp/lib/Commands.h
@@ -91,6 +91,9 @@ class Commands {
static Message deSerializeSingleMessageInBatch(Message& batchedMessage);
+ static SharedBuffer newConsumerStats(proto::BaseCommand& cmd, const std::string& topicName, const std::string& subscriptionName,
+ uint64_t consumerId, uint64_t requestId);
+
private:
Commands();
diff --git a/pulsar-client-cpp/lib/Consumer.cc b/pulsar-client-cpp/lib/Consumer.cc
index 41e163a682476..f05d0035853e2 100644
--- a/pulsar-client-cpp/lib/Consumer.cc
+++ b/pulsar-client-cpp/lib/Consumer.cc
@@ -23,6 +23,50 @@ namespace pulsar {
const std::string EMPTY_STRING;
+BrokerConsumerStats::BrokerConsumerStats():validTill_(now()) {};
+
+BrokerConsumerStats::BrokerConsumerStats(boost::posix_time::ptime& validTill, double msgRateOut, double msgThroughputOut,
+ double msgRateRedeliver, std::string consumerName, int availablePermits,
+ int unackedMessages, bool blockedConsumerOnUnackedMsgs, std::string address,
+ std::string connectedSince, std::string type, double msgRateExpired, long msgBacklog):
+ validTill_(validTill),
+ msgRateOut_(msgRateOut),
+ msgThroughputOut_(msgThroughputOut),
+ msgRateRedeliver_(msgRateRedeliver),
+ consumerName_(consumerName),
+ availablePermits_(availablePermits),
+ unackedMessages_(unackedMessages),
+ blockedConsumerOnUnackedMsgs_(blockedConsumerOnUnackedMsgs),
+ address_(address),
+ connectedSince_(connectedSince),
+ type_(type),
+ msgRateExpired_(msgRateExpired),
+ msgBacklog_(msgBacklog)
+{}
+
+bool BrokerConsumerStats::isValid() const {
+ return now() <= validTill_;
+}
+
+std::ostream& operator<<(std::ostream& os, const BrokerConsumerStats& obj) {
+ os << "\nBrokerConsumerStats ["
+ << "validTill_ = " << obj.validTill_
+ << ", msgRateOut_ = " << obj.msgRateOut_
+ << ", msgThroughputOut_ = " << obj.msgThroughputOut_
+ << ", msgRateRedeliver_ = " << obj.msgRateRedeliver_
+ << ", consumerName_ = " << obj.consumerName_
+ << ", availablePermits_ = " << obj.availablePermits_
+ << ", unackedMessages_ = " << obj.unackedMessages_
+ << ", blockedConsumerOnUnackedMsgs_ = " << obj.blockedConsumerOnUnackedMsgs_
+ << ", address_ = " << obj.address_
+ << ", connectedSince_ = " << obj.connectedSince_
+ << ", type_ = " << obj.type_
+ << ", msgRateExpired_ = " << obj.msgRateExpired_
+ << ", msgBacklog_ = " << obj.msgBacklog_
+ << "]";
+ return os;
+}
+
struct ConsumerConfiguration::Impl {
long unAckedMessagesTimeoutMs;
ConsumerType consumerType;
@@ -259,4 +303,11 @@ void Consumer::redeliverUnacknowledgedMessages() {
impl_->redeliverUnacknowledgedMessages();
}
}
+
+Result Consumer::getConsumerStats(BrokerConsumerStats& BrokerConsumerStats, int partitionIndex) {
+ if (!impl_) {
+ return ResultConsumerNotInitialized;
+ }
+ return impl_->getConsumerStats(BrokerConsumerStats, partitionIndex);
+}
}
diff --git a/pulsar-client-cpp/lib/ConsumerImpl.cc b/pulsar-client-cpp/lib/ConsumerImpl.cc
index f481248c5633f..3f9b8541d8eca 100644
--- a/pulsar-client-cpp/lib/ConsumerImpl.cc
+++ b/pulsar-client-cpp/lib/ConsumerImpl.cc
@@ -50,7 +50,8 @@ ConsumerImpl::ConsumerImpl(const ClientImplPtr client, const std::string& topic,
partitionIndex_(-1),
consumerCreatedPromise_(),
messageListenerRunning_(true),
- batchAcknowledgementTracker_(topic_, subscription, (long)consumerId_) {
+ batchAcknowledgementTracker_(topic_, subscription, (long)consumerId_),
+ brokerConsumerStats_() {
std::stringstream consumerStrStream;
consumerStrStream << "[" << topic_ << ", " << subscription_ << ", " << consumerId_ << "] ";
consumerStr_ = consumerStrStream.str();
@@ -113,7 +114,7 @@ void ConsumerImpl::connectionOpened(const ClientConnectionPtr& cnx) {
lock.unlock();
ClientImplPtr client = client_.lock();
- int requestId = client->newRequestId();
+ uint64_t requestId = client->newRequestId();
SharedBuffer cmd = Commands::newSubscribe(topic_, subscription_, consumerId_, requestId,
getSubType(), consumerName_);
cnx->sendRequestWithId(cmd, requestId).addListener(
@@ -483,13 +484,13 @@ inline proto::CommandSubscribe_SubType ConsumerImpl::getSubType() {
ConsumerType type = config_.getConsumerType();
switch (type) {
case ConsumerExclusive:
- return proto::CommandSubscribe_SubType_Exclusive;
+ return proto::CommandSubscribe::Exclusive;
case ConsumerShared:
- return proto::CommandSubscribe_SubType_Shared;
+ return proto::CommandSubscribe::Shared;
case ConsumerFailover:
- return proto::CommandSubscribe_SubType_Failover;
+ return proto::CommandSubscribe::Failover;
}
}
@@ -686,4 +687,44 @@ int ConsumerImpl::getNumOfPrefetchedMessages() const {
return incomingMessages_.size();
}
+Result ConsumerImpl::getConsumerStats(BrokerConsumerStats& brokerConsumerStats, int partitionIndex) {
+ if (partitionIndex != -1) {
+ LOG_WARN(getName() << "Ignoring the partitionIndex since the topic is not partitioned")
+ }
+
+ if (!isOpen()) {
+ LOG_ERROR(getName() << "Client connection is not open, please try again later.")
+ return ResultConsumerNotInitialized;
+ }
+
+ if (brokerConsumerStats_.isValid()) {
+ LOG_DEBUG(getName() << "Serving data from cache");
+ brokerConsumerStats = brokerConsumerStats_;
+ return ResultOk;
+ }
+
+
+ ClientConnectionPtr cnx = getCnx().lock();
+ if (cnx) {
+ if (cnx->getServerProtocolVersion() >= proto::v7) {
+ ClientImplPtr client = client_.lock();
+ uint64_t requestId = client->newRequestId();
+ LOG_DEBUG(getName() <<
+ " Sending ConsumerStats Command for Consumer - " << getConsumerId() << ", requestId - "<newConsumerStats(topic_, subscription_, consumerId_, requestId).get(consumerStats);
+ if (res == ResultOk) {
+ brokerConsumerStats = brokerConsumerStats_ = consumerStats;
+ }
+ return res;
+ } else {
+ LOG_ERROR(getName() << " Operation not supported since server protobuf version " << cnx->getServerProtocolVersion() << " is older than proto::v7");
+ return ResultOperationNotSupported;
+ }
+ }
+ LOG_ERROR(getName() << " Client Connection not ready for Consumer");
+ return ResultNotConnected;
+}
+
} /* namespace pulsar */
diff --git a/pulsar-client-cpp/lib/ConsumerImpl.h b/pulsar-client-cpp/lib/ConsumerImpl.h
index bbc0b4033f5a1..15ba7539ec4ff 100644
--- a/pulsar-client-cpp/lib/ConsumerImpl.h
+++ b/pulsar-client-cpp/lib/ConsumerImpl.h
@@ -91,6 +91,7 @@ enum ConsumerTopicType {
virtual Result pauseMessageListener();
virtual Result resumeMessageListener();
virtual void redeliverUnacknowledgedMessages();
+ virtual Result getConsumerStats(BrokerConsumerStats& brokerConsumerStats, int partitionIndex = -1);
protected:
void connectionOpened(const ClientConnectionPtr& cnx);
void connectionFailed(Result result);
@@ -103,7 +104,7 @@ enum ConsumerTopicType {
}
virtual const std::string& getName() const;
virtual int getNumOfPrefetchedMessages() const ;
- private:
+private:
bool waitingForZeroQueueSizeMessage;
bool uncompressMessageIfNeeded(const ClientConnectionPtr& cnx, const proto::CommandMessage& msg,
const proto::MessageMetadata& metadata, SharedBuffer& payload);
@@ -133,7 +134,7 @@ enum ConsumerTopicType {
CompressionCodecProvider compressionCodecProvider_;
UnAckedMessageTrackerScopedPtr unAckedMessageTrackerPtr_;
BatchAcknowledgementTracker batchAcknowledgementTracker_;
-
+ BrokerConsumerStats brokerConsumerStats_;
};
} /* namespace pulsar */
diff --git a/pulsar-client-cpp/lib/ConsumerImplBase.h b/pulsar-client-cpp/lib/ConsumerImplBase.h
index 15ca683df8fc3..5d48317f8fdc4 100644
--- a/pulsar-client-cpp/lib/ConsumerImplBase.h
+++ b/pulsar-client-cpp/lib/ConsumerImplBase.h
@@ -47,6 +47,7 @@ class ConsumerImplBase {
virtual void redeliverUnacknowledgedMessages() = 0;
virtual const std::string& getName() const = 0;
virtual int getNumOfPrefetchedMessages() const = 0;
+ virtual Result getConsumerStats(BrokerConsumerStats& brokerConsumerStats, int partitionIndex = -1) = 0;
};
}
#endif //PULSAR_CONSUMER_IMPL_BASE_HEADER
diff --git a/pulsar-client-cpp/lib/PartitionedConsumerImpl.cc b/pulsar-client-cpp/lib/PartitionedConsumerImpl.cc
index 7b54da730e988..5de6993902c5c 100644
--- a/pulsar-client-cpp/lib/PartitionedConsumerImpl.cc
+++ b/pulsar-client-cpp/lib/PartitionedConsumerImpl.cc
@@ -380,4 +380,12 @@ namespace pulsar {
return messages_.size();
}
+ Result PartitionedConsumerImpl::getConsumerStats(BrokerConsumerStats& BrokerConsumerStats, int partitionIndex) {
+ if (partitionIndex >= numPartitions_ && partitionIndex < 0 && consumers_.size() <= partitionIndex)
+ {
+ LOG_ERROR(getName() << " PartitionIndex must be positive and less than number of partitiones")
+ return ResultInvalidConfiguration;
+ }
+ return consumers_[partitionIndex]->getConsumerStats(BrokerConsumerStats);
+ }
}
diff --git a/pulsar-client-cpp/lib/PartitionedConsumerImpl.h b/pulsar-client-cpp/lib/PartitionedConsumerImpl.h
index ff6fc4563ba0a..3338adda125a2 100644
--- a/pulsar-client-cpp/lib/PartitionedConsumerImpl.h
+++ b/pulsar-client-cpp/lib/PartitionedConsumerImpl.h
@@ -60,6 +60,7 @@ namespace pulsar {
virtual void redeliverUnacknowledgedMessages();
virtual const std::string& getName() const;
virtual int getNumOfPrefetchedMessages() const ;
+ virtual Result getConsumerStats(BrokerConsumerStats& brokerConsumerStats, int partitionIndex);
private:
const ClientImplPtr client_;
const std::string subscriptionName_;
diff --git a/pulsar-client-cpp/lib/PulsarApi.pb.cc b/pulsar-client-cpp/lib/PulsarApi.pb.cc
index b415698764435..a67bb559d8b7f 100644
--- a/pulsar-client-cpp/lib/PulsarApi.pb.cc
+++ b/pulsar-client-cpp/lib/PulsarApi.pb.cc
@@ -44,6 +44,8 @@ void protobuf_ShutdownFile_PulsarApi_2eproto() {
delete CommandError::default_instance_;
delete CommandPing::default_instance_;
delete CommandPong::default_instance_;
+ delete CommandConsumerStats::default_instance_;
+ delete CommandConsumerStatsResponse::default_instance_;
delete BaseCommand::default_instance_;
}
@@ -86,6 +88,8 @@ void protobuf_AddDesc_PulsarApi_2eproto() {
CommandError::default_instance_ = new CommandError();
CommandPing::default_instance_ = new CommandPing();
CommandPong::default_instance_ = new CommandPong();
+ CommandConsumerStats::default_instance_ = new CommandConsumerStats();
+ CommandConsumerStatsResponse::default_instance_ = new CommandConsumerStatsResponse();
BaseCommand::default_instance_ = new BaseCommand();
MessageIdData::default_instance_->InitAsDefaultInstance();
KeyValue::default_instance_->InitAsDefaultInstance();
@@ -114,6 +118,8 @@ void protobuf_AddDesc_PulsarApi_2eproto() {
CommandError::default_instance_->InitAsDefaultInstance();
CommandPing::default_instance_->InitAsDefaultInstance();
CommandPong::default_instance_->InitAsDefaultInstance();
+ CommandConsumerStats::default_instance_->InitAsDefaultInstance();
+ CommandConsumerStatsResponse::default_instance_->InitAsDefaultInstance();
BaseCommand::default_instance_->InitAsDefaultInstance();
::google::protobuf::internal::OnShutdown(&protobuf_ShutdownFile_PulsarApi_2eproto);
}
@@ -155,6 +161,10 @@ bool ServerError_IsValid(int value) {
case 7:
case 8:
case 9:
+ case 10:
+ case 11:
+ case 12:
+ case 13:
return true;
default:
return false;
@@ -181,6 +191,7 @@ bool ProtocolVersion_IsValid(int value) {
case 4:
case 5:
case 6:
+ case 7:
return true;
default:
return false;
@@ -7917,272 +7928,1314 @@ ::std::string CommandPong::GetTypeName() const {
// ===================================================================
-bool BaseCommand_Type_IsValid(int value) {
- switch(value) {
- case 2:
- case 3:
- case 4:
- case 5:
- case 6:
- case 7:
- case 8:
- case 9:
- case 10:
- case 11:
- case 12:
- case 13:
- case 14:
- case 15:
- case 16:
- case 17:
- case 18:
- case 19:
- case 20:
- case 21:
- case 22:
- case 23:
- case 24:
- return true;
- default:
- return false;
- }
-}
-
-#ifndef _MSC_VER
-const BaseCommand_Type BaseCommand::CONNECT;
-const BaseCommand_Type BaseCommand::CONNECTED;
-const BaseCommand_Type BaseCommand::SUBSCRIBE;
-const BaseCommand_Type BaseCommand::PRODUCER;
-const BaseCommand_Type BaseCommand::SEND;
-const BaseCommand_Type BaseCommand::SEND_RECEIPT;
-const BaseCommand_Type BaseCommand::SEND_ERROR;
-const BaseCommand_Type BaseCommand::MESSAGE;
-const BaseCommand_Type BaseCommand::ACK;
-const BaseCommand_Type BaseCommand::FLOW;
-const BaseCommand_Type BaseCommand::UNSUBSCRIBE;
-const BaseCommand_Type BaseCommand::SUCCESS;
-const BaseCommand_Type BaseCommand::ERROR;
-const BaseCommand_Type BaseCommand::CLOSE_PRODUCER;
-const BaseCommand_Type BaseCommand::CLOSE_CONSUMER;
-const BaseCommand_Type BaseCommand::PRODUCER_SUCCESS;
-const BaseCommand_Type BaseCommand::PING;
-const BaseCommand_Type BaseCommand::PONG;
-const BaseCommand_Type BaseCommand::REDELIVER_UNACKNOWLEDGED_MESSAGES;
-const BaseCommand_Type BaseCommand::PARTITIONED_METADATA;
-const BaseCommand_Type BaseCommand::PARTITIONED_METADATA_RESPONSE;
-const BaseCommand_Type BaseCommand::LOOKUP;
-const BaseCommand_Type BaseCommand::LOOKUP_RESPONSE;
-const BaseCommand_Type BaseCommand::Type_MIN;
-const BaseCommand_Type BaseCommand::Type_MAX;
-const int BaseCommand::Type_ARRAYSIZE;
-#endif // _MSC_VER
#ifndef _MSC_VER
-const int BaseCommand::kTypeFieldNumber;
-const int BaseCommand::kConnectFieldNumber;
-const int BaseCommand::kConnectedFieldNumber;
-const int BaseCommand::kSubscribeFieldNumber;
-const int BaseCommand::kProducerFieldNumber;
-const int BaseCommand::kSendFieldNumber;
-const int BaseCommand::kSendReceiptFieldNumber;
-const int BaseCommand::kSendErrorFieldNumber;
-const int BaseCommand::kMessageFieldNumber;
-const int BaseCommand::kAckFieldNumber;
-const int BaseCommand::kFlowFieldNumber;
-const int BaseCommand::kUnsubscribeFieldNumber;
-const int BaseCommand::kSuccessFieldNumber;
-const int BaseCommand::kErrorFieldNumber;
-const int BaseCommand::kCloseProducerFieldNumber;
-const int BaseCommand::kCloseConsumerFieldNumber;
-const int BaseCommand::kProducerSuccessFieldNumber;
-const int BaseCommand::kPingFieldNumber;
-const int BaseCommand::kPongFieldNumber;
-const int BaseCommand::kRedeliverUnacknowledgedMessagesFieldNumber;
-const int BaseCommand::kPartitionMetadataFieldNumber;
-const int BaseCommand::kPartitionMetadataResponseFieldNumber;
-const int BaseCommand::kLookupTopicFieldNumber;
-const int BaseCommand::kLookupTopicResponseFieldNumber;
+const int CommandConsumerStats::kRequestIdFieldNumber;
+const int CommandConsumerStats::kTopicNameFieldNumber;
+const int CommandConsumerStats::kSubscriptionNameFieldNumber;
+const int CommandConsumerStats::kConsumerIdFieldNumber;
#endif // !_MSC_VER
-BaseCommand::BaseCommand()
+CommandConsumerStats::CommandConsumerStats()
: ::google::protobuf::MessageLite() {
SharedCtor();
- // @@protoc_insertion_point(constructor:pulsar.proto.BaseCommand)
+ // @@protoc_insertion_point(constructor:pulsar.proto.CommandConsumerStats)
}
-void BaseCommand::InitAsDefaultInstance() {
-#ifdef GOOGLE_PROTOBUF_NO_STATIC_INITIALIZER
- connect_ = const_cast< ::pulsar::proto::CommandConnect*>(
- ::pulsar::proto::CommandConnect::internal_default_instance());
-#else
- connect_ = const_cast< ::pulsar::proto::CommandConnect*>(&::pulsar::proto::CommandConnect::default_instance());
-#endif
-#ifdef GOOGLE_PROTOBUF_NO_STATIC_INITIALIZER
- connected_ = const_cast< ::pulsar::proto::CommandConnected*>(
- ::pulsar::proto::CommandConnected::internal_default_instance());
-#else
- connected_ = const_cast< ::pulsar::proto::CommandConnected*>(&::pulsar::proto::CommandConnected::default_instance());
-#endif
-#ifdef GOOGLE_PROTOBUF_NO_STATIC_INITIALIZER
- subscribe_ = const_cast< ::pulsar::proto::CommandSubscribe*>(
- ::pulsar::proto::CommandSubscribe::internal_default_instance());
-#else
- subscribe_ = const_cast< ::pulsar::proto::CommandSubscribe*>(&::pulsar::proto::CommandSubscribe::default_instance());
-#endif
-#ifdef GOOGLE_PROTOBUF_NO_STATIC_INITIALIZER
- producer_ = const_cast< ::pulsar::proto::CommandProducer*>(
- ::pulsar::proto::CommandProducer::internal_default_instance());
-#else
- producer_ = const_cast< ::pulsar::proto::CommandProducer*>(&::pulsar::proto::CommandProducer::default_instance());
-#endif
-#ifdef GOOGLE_PROTOBUF_NO_STATIC_INITIALIZER
- send_ = const_cast< ::pulsar::proto::CommandSend*>(
- ::pulsar::proto::CommandSend::internal_default_instance());
-#else
- send_ = const_cast< ::pulsar::proto::CommandSend*>(&::pulsar::proto::CommandSend::default_instance());
-#endif
-#ifdef GOOGLE_PROTOBUF_NO_STATIC_INITIALIZER
- send_receipt_ = const_cast< ::pulsar::proto::CommandSendReceipt*>(
- ::pulsar::proto::CommandSendReceipt::internal_default_instance());
-#else
- send_receipt_ = const_cast< ::pulsar::proto::CommandSendReceipt*>(&::pulsar::proto::CommandSendReceipt::default_instance());
-#endif
-#ifdef GOOGLE_PROTOBUF_NO_STATIC_INITIALIZER
- send_error_ = const_cast< ::pulsar::proto::CommandSendError*>(
- ::pulsar::proto::CommandSendError::internal_default_instance());
-#else
- send_error_ = const_cast< ::pulsar::proto::CommandSendError*>(&::pulsar::proto::CommandSendError::default_instance());
-#endif
-#ifdef GOOGLE_PROTOBUF_NO_STATIC_INITIALIZER
- message_ = const_cast< ::pulsar::proto::CommandMessage*>(
- ::pulsar::proto::CommandMessage::internal_default_instance());
-#else
- message_ = const_cast< ::pulsar::proto::CommandMessage*>(&::pulsar::proto::CommandMessage::default_instance());
-#endif
-#ifdef GOOGLE_PROTOBUF_NO_STATIC_INITIALIZER
- ack_ = const_cast< ::pulsar::proto::CommandAck*>(
- ::pulsar::proto::CommandAck::internal_default_instance());
-#else
- ack_ = const_cast< ::pulsar::proto::CommandAck*>(&::pulsar::proto::CommandAck::default_instance());
-#endif
-#ifdef GOOGLE_PROTOBUF_NO_STATIC_INITIALIZER
- flow_ = const_cast< ::pulsar::proto::CommandFlow*>(
- ::pulsar::proto::CommandFlow::internal_default_instance());
-#else
- flow_ = const_cast< ::pulsar::proto::CommandFlow*>(&::pulsar::proto::CommandFlow::default_instance());
-#endif
-#ifdef GOOGLE_PROTOBUF_NO_STATIC_INITIALIZER
- unsubscribe_ = const_cast< ::pulsar::proto::CommandUnsubscribe*>(
- ::pulsar::proto::CommandUnsubscribe::internal_default_instance());
-#else
- unsubscribe_ = const_cast< ::pulsar::proto::CommandUnsubscribe*>(&::pulsar::proto::CommandUnsubscribe::default_instance());
-#endif
-#ifdef GOOGLE_PROTOBUF_NO_STATIC_INITIALIZER
- success_ = const_cast< ::pulsar::proto::CommandSuccess*>(
- ::pulsar::proto::CommandSuccess::internal_default_instance());
-#else
- success_ = const_cast< ::pulsar::proto::CommandSuccess*>(&::pulsar::proto::CommandSuccess::default_instance());
-#endif
-#ifdef GOOGLE_PROTOBUF_NO_STATIC_INITIALIZER
- error_ = const_cast< ::pulsar::proto::CommandError*>(
- ::pulsar::proto::CommandError::internal_default_instance());
-#else
- error_ = const_cast< ::pulsar::proto::CommandError*>(&::pulsar::proto::CommandError::default_instance());
-#endif
-#ifdef GOOGLE_PROTOBUF_NO_STATIC_INITIALIZER
- close_producer_ = const_cast< ::pulsar::proto::CommandCloseProducer*>(
- ::pulsar::proto::CommandCloseProducer::internal_default_instance());
-#else
- close_producer_ = const_cast< ::pulsar::proto::CommandCloseProducer*>(&::pulsar::proto::CommandCloseProducer::default_instance());
-#endif
-#ifdef GOOGLE_PROTOBUF_NO_STATIC_INITIALIZER
- close_consumer_ = const_cast< ::pulsar::proto::CommandCloseConsumer*>(
- ::pulsar::proto::CommandCloseConsumer::internal_default_instance());
-#else
- close_consumer_ = const_cast< ::pulsar::proto::CommandCloseConsumer*>(&::pulsar::proto::CommandCloseConsumer::default_instance());
-#endif
-#ifdef GOOGLE_PROTOBUF_NO_STATIC_INITIALIZER
- producer_success_ = const_cast< ::pulsar::proto::CommandProducerSuccess*>(
- ::pulsar::proto::CommandProducerSuccess::internal_default_instance());
-#else
- producer_success_ = const_cast< ::pulsar::proto::CommandProducerSuccess*>(&::pulsar::proto::CommandProducerSuccess::default_instance());
-#endif
-#ifdef GOOGLE_PROTOBUF_NO_STATIC_INITIALIZER
- ping_ = const_cast< ::pulsar::proto::CommandPing*>(
- ::pulsar::proto::CommandPing::internal_default_instance());
-#else
- ping_ = const_cast< ::pulsar::proto::CommandPing*>(&::pulsar::proto::CommandPing::default_instance());
-#endif
-#ifdef GOOGLE_PROTOBUF_NO_STATIC_INITIALIZER
- pong_ = const_cast< ::pulsar::proto::CommandPong*>(
- ::pulsar::proto::CommandPong::internal_default_instance());
-#else
- pong_ = const_cast< ::pulsar::proto::CommandPong*>(&::pulsar::proto::CommandPong::default_instance());
-#endif
-#ifdef GOOGLE_PROTOBUF_NO_STATIC_INITIALIZER
- redeliverunacknowledgedmessages_ = const_cast< ::pulsar::proto::CommandRedeliverUnacknowledgedMessages*>(
- ::pulsar::proto::CommandRedeliverUnacknowledgedMessages::internal_default_instance());
-#else
- redeliverunacknowledgedmessages_ = const_cast< ::pulsar::proto::CommandRedeliverUnacknowledgedMessages*>(&::pulsar::proto::CommandRedeliverUnacknowledgedMessages::default_instance());
-#endif
-#ifdef GOOGLE_PROTOBUF_NO_STATIC_INITIALIZER
- partitionmetadata_ = const_cast< ::pulsar::proto::CommandPartitionedTopicMetadata*>(
- ::pulsar::proto::CommandPartitionedTopicMetadata::internal_default_instance());
-#else
- partitionmetadata_ = const_cast< ::pulsar::proto::CommandPartitionedTopicMetadata*>(&::pulsar::proto::CommandPartitionedTopicMetadata::default_instance());
-#endif
-#ifdef GOOGLE_PROTOBUF_NO_STATIC_INITIALIZER
- partitionmetadataresponse_ = const_cast< ::pulsar::proto::CommandPartitionedTopicMetadataResponse*>(
- ::pulsar::proto::CommandPartitionedTopicMetadataResponse::internal_default_instance());
-#else
- partitionmetadataresponse_ = const_cast< ::pulsar::proto::CommandPartitionedTopicMetadataResponse*>(&::pulsar::proto::CommandPartitionedTopicMetadataResponse::default_instance());
-#endif
-#ifdef GOOGLE_PROTOBUF_NO_STATIC_INITIALIZER
- lookuptopic_ = const_cast< ::pulsar::proto::CommandLookupTopic*>(
- ::pulsar::proto::CommandLookupTopic::internal_default_instance());
-#else
- lookuptopic_ = const_cast< ::pulsar::proto::CommandLookupTopic*>(&::pulsar::proto::CommandLookupTopic::default_instance());
-#endif
-#ifdef GOOGLE_PROTOBUF_NO_STATIC_INITIALIZER
- lookuptopicresponse_ = const_cast< ::pulsar::proto::CommandLookupTopicResponse*>(
- ::pulsar::proto::CommandLookupTopicResponse::internal_default_instance());
-#else
- lookuptopicresponse_ = const_cast< ::pulsar::proto::CommandLookupTopicResponse*>(&::pulsar::proto::CommandLookupTopicResponse::default_instance());
-#endif
+void CommandConsumerStats::InitAsDefaultInstance() {
}
-BaseCommand::BaseCommand(const BaseCommand& from)
+CommandConsumerStats::CommandConsumerStats(const CommandConsumerStats& from)
: ::google::protobuf::MessageLite() {
SharedCtor();
MergeFrom(from);
- // @@protoc_insertion_point(copy_constructor:pulsar.proto.BaseCommand)
+ // @@protoc_insertion_point(copy_constructor:pulsar.proto.CommandConsumerStats)
}
-void BaseCommand::SharedCtor() {
+void CommandConsumerStats::SharedCtor() {
+ ::google::protobuf::internal::GetEmptyString();
_cached_size_ = 0;
- type_ = 2;
- connect_ = NULL;
- connected_ = NULL;
- subscribe_ = NULL;
- producer_ = NULL;
- send_ = NULL;
- send_receipt_ = NULL;
- send_error_ = NULL;
- message_ = NULL;
- ack_ = NULL;
- flow_ = NULL;
- unsubscribe_ = NULL;
- success_ = NULL;
- error_ = NULL;
- close_producer_ = NULL;
- close_consumer_ = NULL;
- producer_success_ = NULL;
- ping_ = NULL;
- pong_ = NULL;
- redeliverunacknowledgedmessages_ = NULL;
- partitionmetadata_ = NULL;
- partitionmetadataresponse_ = NULL;
- lookuptopic_ = NULL;
+ request_id_ = GOOGLE_ULONGLONG(0);
+ topic_name_ = const_cast< ::std::string*>(&::google::protobuf::internal::GetEmptyStringAlreadyInited());
+ subscription_name_ = const_cast< ::std::string*>(&::google::protobuf::internal::GetEmptyStringAlreadyInited());
+ consumer_id_ = GOOGLE_ULONGLONG(0);
+ ::memset(_has_bits_, 0, sizeof(_has_bits_));
+}
+
+CommandConsumerStats::~CommandConsumerStats() {
+ // @@protoc_insertion_point(destructor:pulsar.proto.CommandConsumerStats)
+ SharedDtor();
+}
+
+void CommandConsumerStats::SharedDtor() {
+ if (topic_name_ != &::google::protobuf::internal::GetEmptyStringAlreadyInited()) {
+ delete topic_name_;
+ }
+ if (subscription_name_ != &::google::protobuf::internal::GetEmptyStringAlreadyInited()) {
+ delete subscription_name_;
+ }
+ #ifdef GOOGLE_PROTOBUF_NO_STATIC_INITIALIZER
+ if (this != &default_instance()) {
+ #else
+ if (this != default_instance_) {
+ #endif
+ }
+}
+
+void CommandConsumerStats::SetCachedSize(int size) const {
+ GOOGLE_SAFE_CONCURRENT_WRITES_BEGIN();
+ _cached_size_ = size;
+ GOOGLE_SAFE_CONCURRENT_WRITES_END();
+}
+const CommandConsumerStats& CommandConsumerStats::default_instance() {
+#ifdef GOOGLE_PROTOBUF_NO_STATIC_INITIALIZER
+ protobuf_AddDesc_PulsarApi_2eproto();
+#else
+ if (default_instance_ == NULL) protobuf_AddDesc_PulsarApi_2eproto();
+#endif
+ return *default_instance_;
+}
+
+CommandConsumerStats* CommandConsumerStats::default_instance_ = NULL;
+
+CommandConsumerStats* CommandConsumerStats::New() const {
+ return new CommandConsumerStats;
+}
+
+void CommandConsumerStats::Clear() {
+ if (_has_bits_[0 / 32] & 15) {
+ request_id_ = GOOGLE_ULONGLONG(0);
+ if (has_topic_name()) {
+ if (topic_name_ != &::google::protobuf::internal::GetEmptyStringAlreadyInited()) {
+ topic_name_->clear();
+ }
+ }
+ if (has_subscription_name()) {
+ if (subscription_name_ != &::google::protobuf::internal::GetEmptyStringAlreadyInited()) {
+ subscription_name_->clear();
+ }
+ }
+ consumer_id_ = GOOGLE_ULONGLONG(0);
+ }
+ ::memset(_has_bits_, 0, sizeof(_has_bits_));
+ mutable_unknown_fields()->clear();
+}
+
+bool CommandConsumerStats::MergePartialFromCodedStream(
+ ::google::protobuf::io::CodedInputStream* input) {
+#define DO_(EXPRESSION) if (!(EXPRESSION)) goto failure
+ ::google::protobuf::uint32 tag;
+ ::google::protobuf::io::StringOutputStream unknown_fields_string(
+ mutable_unknown_fields());
+ ::google::protobuf::io::CodedOutputStream unknown_fields_stream(
+ &unknown_fields_string);
+ // @@protoc_insertion_point(parse_start:pulsar.proto.CommandConsumerStats)
+ for (;;) {
+ ::std::pair< ::google::protobuf::uint32, bool> p = input->ReadTagWithCutoff(127);
+ tag = p.first;
+ if (!p.second) goto handle_unusual;
+ switch (::google::protobuf::internal::WireFormatLite::GetTagFieldNumber(tag)) {
+ // required uint64 request_id = 1;
+ case 1: {
+ if (tag == 8) {
+ DO_((::google::protobuf::internal::WireFormatLite::ReadPrimitive<
+ ::google::protobuf::uint64, ::google::protobuf::internal::WireFormatLite::TYPE_UINT64>(
+ input, &request_id_)));
+ set_has_request_id();
+ } else {
+ goto handle_unusual;
+ }
+ if (input->ExpectTag(18)) goto parse_topic_name;
+ break;
+ }
+
+ // required string topic_name = 2;
+ case 2: {
+ if (tag == 18) {
+ parse_topic_name:
+ DO_(::google::protobuf::internal::WireFormatLite::ReadString(
+ input, this->mutable_topic_name()));
+ } else {
+ goto handle_unusual;
+ }
+ if (input->ExpectTag(26)) goto parse_subscription_name;
+ break;
+ }
+
+ // required string subscription_name = 3;
+ case 3: {
+ if (tag == 26) {
+ parse_subscription_name:
+ DO_(::google::protobuf::internal::WireFormatLite::ReadString(
+ input, this->mutable_subscription_name()));
+ } else {
+ goto handle_unusual;
+ }
+ if (input->ExpectTag(32)) goto parse_consumer_id;
+ break;
+ }
+
+ // required uint64 consumer_id = 4;
+ case 4: {
+ if (tag == 32) {
+ parse_consumer_id:
+ DO_((::google::protobuf::internal::WireFormatLite::ReadPrimitive<
+ ::google::protobuf::uint64, ::google::protobuf::internal::WireFormatLite::TYPE_UINT64>(
+ input, &consumer_id_)));
+ set_has_consumer_id();
+ } else {
+ goto handle_unusual;
+ }
+ if (input->ExpectAtEnd()) goto success;
+ break;
+ }
+
+ default: {
+ handle_unusual:
+ if (tag == 0 ||
+ ::google::protobuf::internal::WireFormatLite::GetTagWireType(tag) ==
+ ::google::protobuf::internal::WireFormatLite::WIRETYPE_END_GROUP) {
+ goto success;
+ }
+ DO_(::google::protobuf::internal::WireFormatLite::SkipField(
+ input, tag, &unknown_fields_stream));
+ break;
+ }
+ }
+ }
+success:
+ // @@protoc_insertion_point(parse_success:pulsar.proto.CommandConsumerStats)
+ return true;
+failure:
+ // @@protoc_insertion_point(parse_failure:pulsar.proto.CommandConsumerStats)
+ return false;
+#undef DO_
+}
+
+void CommandConsumerStats::SerializeWithCachedSizes(
+ ::google::protobuf::io::CodedOutputStream* output) const {
+ // @@protoc_insertion_point(serialize_start:pulsar.proto.CommandConsumerStats)
+ // required uint64 request_id = 1;
+ if (has_request_id()) {
+ ::google::protobuf::internal::WireFormatLite::WriteUInt64(1, this->request_id(), output);
+ }
+
+ // required string topic_name = 2;
+ if (has_topic_name()) {
+ ::google::protobuf::internal::WireFormatLite::WriteStringMaybeAliased(
+ 2, this->topic_name(), output);
+ }
+
+ // required string subscription_name = 3;
+ if (has_subscription_name()) {
+ ::google::protobuf::internal::WireFormatLite::WriteStringMaybeAliased(
+ 3, this->subscription_name(), output);
+ }
+
+ // required uint64 consumer_id = 4;
+ if (has_consumer_id()) {
+ ::google::protobuf::internal::WireFormatLite::WriteUInt64(4, this->consumer_id(), output);
+ }
+
+ output->WriteRaw(unknown_fields().data(),
+ unknown_fields().size());
+ // @@protoc_insertion_point(serialize_end:pulsar.proto.CommandConsumerStats)
+}
+
+int CommandConsumerStats::ByteSize() const {
+ int total_size = 0;
+
+ if (_has_bits_[0 / 32] & (0xffu << (0 % 32))) {
+ // required uint64 request_id = 1;
+ if (has_request_id()) {
+ total_size += 1 +
+ ::google::protobuf::internal::WireFormatLite::UInt64Size(
+ this->request_id());
+ }
+
+ // required string topic_name = 2;
+ if (has_topic_name()) {
+ total_size += 1 +
+ ::google::protobuf::internal::WireFormatLite::StringSize(
+ this->topic_name());
+ }
+
+ // required string subscription_name = 3;
+ if (has_subscription_name()) {
+ total_size += 1 +
+ ::google::protobuf::internal::WireFormatLite::StringSize(
+ this->subscription_name());
+ }
+
+ // required uint64 consumer_id = 4;
+ if (has_consumer_id()) {
+ total_size += 1 +
+ ::google::protobuf::internal::WireFormatLite::UInt64Size(
+ this->consumer_id());
+ }
+
+ }
+ total_size += unknown_fields().size();
+
+ GOOGLE_SAFE_CONCURRENT_WRITES_BEGIN();
+ _cached_size_ = total_size;
+ GOOGLE_SAFE_CONCURRENT_WRITES_END();
+ return total_size;
+}
+
+void CommandConsumerStats::CheckTypeAndMergeFrom(
+ const ::google::protobuf::MessageLite& from) {
+ MergeFrom(*::google::protobuf::down_cast(&from));
+}
+
+void CommandConsumerStats::MergeFrom(const CommandConsumerStats& from) {
+ GOOGLE_CHECK_NE(&from, this);
+ if (from._has_bits_[0 / 32] & (0xffu << (0 % 32))) {
+ if (from.has_request_id()) {
+ set_request_id(from.request_id());
+ }
+ if (from.has_topic_name()) {
+ set_topic_name(from.topic_name());
+ }
+ if (from.has_subscription_name()) {
+ set_subscription_name(from.subscription_name());
+ }
+ if (from.has_consumer_id()) {
+ set_consumer_id(from.consumer_id());
+ }
+ }
+ mutable_unknown_fields()->append(from.unknown_fields());
+}
+
+void CommandConsumerStats::CopyFrom(const CommandConsumerStats& from) {
+ if (&from == this) return;
+ Clear();
+ MergeFrom(from);
+}
+
+bool CommandConsumerStats::IsInitialized() const {
+ if ((_has_bits_[0] & 0x0000000f) != 0x0000000f) return false;
+
+ return true;
+}
+
+void CommandConsumerStats::Swap(CommandConsumerStats* other) {
+ if (other != this) {
+ std::swap(request_id_, other->request_id_);
+ std::swap(topic_name_, other->topic_name_);
+ std::swap(subscription_name_, other->subscription_name_);
+ std::swap(consumer_id_, other->consumer_id_);
+ std::swap(_has_bits_[0], other->_has_bits_[0]);
+ _unknown_fields_.swap(other->_unknown_fields_);
+ std::swap(_cached_size_, other->_cached_size_);
+ }
+}
+
+::std::string CommandConsumerStats::GetTypeName() const {
+ return "pulsar.proto.CommandConsumerStats";
+}
+
+
+// ===================================================================
+
+#ifndef _MSC_VER
+const int CommandConsumerStatsResponse::kRequestIdFieldNumber;
+const int CommandConsumerStatsResponse::kErrorCodeFieldNumber;
+const int CommandConsumerStatsResponse::kErrorMessageFieldNumber;
+const int CommandConsumerStatsResponse::kMsgRateOutFieldNumber;
+const int CommandConsumerStatsResponse::kMsgThroughputOutFieldNumber;
+const int CommandConsumerStatsResponse::kMsgRateRedeliverFieldNumber;
+const int CommandConsumerStatsResponse::kConsumerNameFieldNumber;
+const int CommandConsumerStatsResponse::kAvailablePermitsFieldNumber;
+const int CommandConsumerStatsResponse::kUnackedMessagesFieldNumber;
+const int CommandConsumerStatsResponse::kBlockedConsumerOnUnackedMsgsFieldNumber;
+const int CommandConsumerStatsResponse::kAddressFieldNumber;
+const int CommandConsumerStatsResponse::kConnectedSinceFieldNumber;
+const int CommandConsumerStatsResponse::kTypeFieldNumber;
+const int CommandConsumerStatsResponse::kMsgRateExpiredFieldNumber;
+const int CommandConsumerStatsResponse::kMsgBacklogFieldNumber;
+#endif // !_MSC_VER
+
+CommandConsumerStatsResponse::CommandConsumerStatsResponse()
+ : ::google::protobuf::MessageLite() {
+ SharedCtor();
+ // @@protoc_insertion_point(constructor:pulsar.proto.CommandConsumerStatsResponse)
+}
+
+void CommandConsumerStatsResponse::InitAsDefaultInstance() {
+}
+
+CommandConsumerStatsResponse::CommandConsumerStatsResponse(const CommandConsumerStatsResponse& from)
+ : ::google::protobuf::MessageLite() {
+ SharedCtor();
+ MergeFrom(from);
+ // @@protoc_insertion_point(copy_constructor:pulsar.proto.CommandConsumerStatsResponse)
+}
+
+void CommandConsumerStatsResponse::SharedCtor() {
+ ::google::protobuf::internal::GetEmptyString();
+ _cached_size_ = 0;
+ request_id_ = GOOGLE_ULONGLONG(0);
+ error_code_ = 0;
+ error_message_ = const_cast< ::std::string*>(&::google::protobuf::internal::GetEmptyStringAlreadyInited());
+ msgrateout_ = 0;
+ msgthroughputout_ = 0;
+ msgrateredeliver_ = 0;
+ consumername_ = const_cast< ::std::string*>(&::google::protobuf::internal::GetEmptyStringAlreadyInited());
+ availablepermits_ = GOOGLE_ULONGLONG(0);
+ unackedmessages_ = GOOGLE_ULONGLONG(0);
+ blockedconsumeronunackedmsgs_ = false;
+ address_ = const_cast< ::std::string*>(&::google::protobuf::internal::GetEmptyStringAlreadyInited());
+ connectedsince_ = const_cast< ::std::string*>(&::google::protobuf::internal::GetEmptyStringAlreadyInited());
+ type_ = const_cast< ::std::string*>(&::google::protobuf::internal::GetEmptyStringAlreadyInited());
+ msgrateexpired_ = 0;
+ msgbacklog_ = GOOGLE_ULONGLONG(0);
+ ::memset(_has_bits_, 0, sizeof(_has_bits_));
+}
+
+CommandConsumerStatsResponse::~CommandConsumerStatsResponse() {
+ // @@protoc_insertion_point(destructor:pulsar.proto.CommandConsumerStatsResponse)
+ SharedDtor();
+}
+
+void CommandConsumerStatsResponse::SharedDtor() {
+ if (error_message_ != &::google::protobuf::internal::GetEmptyStringAlreadyInited()) {
+ delete error_message_;
+ }
+ if (consumername_ != &::google::protobuf::internal::GetEmptyStringAlreadyInited()) {
+ delete consumername_;
+ }
+ if (address_ != &::google::protobuf::internal::GetEmptyStringAlreadyInited()) {
+ delete address_;
+ }
+ if (connectedsince_ != &::google::protobuf::internal::GetEmptyStringAlreadyInited()) {
+ delete connectedsince_;
+ }
+ if (type_ != &::google::protobuf::internal::GetEmptyStringAlreadyInited()) {
+ delete type_;
+ }
+ #ifdef GOOGLE_PROTOBUF_NO_STATIC_INITIALIZER
+ if (this != &default_instance()) {
+ #else
+ if (this != default_instance_) {
+ #endif
+ }
+}
+
+void CommandConsumerStatsResponse::SetCachedSize(int size) const {
+ GOOGLE_SAFE_CONCURRENT_WRITES_BEGIN();
+ _cached_size_ = size;
+ GOOGLE_SAFE_CONCURRENT_WRITES_END();
+}
+const CommandConsumerStatsResponse& CommandConsumerStatsResponse::default_instance() {
+#ifdef GOOGLE_PROTOBUF_NO_STATIC_INITIALIZER
+ protobuf_AddDesc_PulsarApi_2eproto();
+#else
+ if (default_instance_ == NULL) protobuf_AddDesc_PulsarApi_2eproto();
+#endif
+ return *default_instance_;
+}
+
+CommandConsumerStatsResponse* CommandConsumerStatsResponse::default_instance_ = NULL;
+
+CommandConsumerStatsResponse* CommandConsumerStatsResponse::New() const {
+ return new CommandConsumerStatsResponse;
+}
+
+void CommandConsumerStatsResponse::Clear() {
+#define OFFSET_OF_FIELD_(f) (reinterpret_cast( \
+ &reinterpret_cast(16)->f) - \
+ reinterpret_cast(16))
+
+#define ZR_(first, last) do { \
+ size_t f = OFFSET_OF_FIELD_(first); \
+ size_t n = OFFSET_OF_FIELD_(last) - f + sizeof(last); \
+ ::memset(&first, 0, n); \
+ } while (0)
+
+ if (_has_bits_[0 / 32] & 255) {
+ ZR_(msgrateout_, error_code_);
+ request_id_ = GOOGLE_ULONGLONG(0);
+ if (has_error_message()) {
+ if (error_message_ != &::google::protobuf::internal::GetEmptyStringAlreadyInited()) {
+ error_message_->clear();
+ }
+ }
+ if (has_consumername()) {
+ if (consumername_ != &::google::protobuf::internal::GetEmptyStringAlreadyInited()) {
+ consumername_->clear();
+ }
+ }
+ availablepermits_ = GOOGLE_ULONGLONG(0);
+ }
+ if (_has_bits_[8 / 32] & 32512) {
+ ZR_(msgrateexpired_, msgbacklog_);
+ unackedmessages_ = GOOGLE_ULONGLONG(0);
+ blockedconsumeronunackedmsgs_ = false;
+ if (has_address()) {
+ if (address_ != &::google::protobuf::internal::GetEmptyStringAlreadyInited()) {
+ address_->clear();
+ }
+ }
+ if (has_connectedsince()) {
+ if (connectedsince_ != &::google::protobuf::internal::GetEmptyStringAlreadyInited()) {
+ connectedsince_->clear();
+ }
+ }
+ if (has_type()) {
+ if (type_ != &::google::protobuf::internal::GetEmptyStringAlreadyInited()) {
+ type_->clear();
+ }
+ }
+ }
+
+#undef OFFSET_OF_FIELD_
+#undef ZR_
+
+ ::memset(_has_bits_, 0, sizeof(_has_bits_));
+ mutable_unknown_fields()->clear();
+}
+
+bool CommandConsumerStatsResponse::MergePartialFromCodedStream(
+ ::google::protobuf::io::CodedInputStream* input) {
+#define DO_(EXPRESSION) if (!(EXPRESSION)) goto failure
+ ::google::protobuf::uint32 tag;
+ ::google::protobuf::io::StringOutputStream unknown_fields_string(
+ mutable_unknown_fields());
+ ::google::protobuf::io::CodedOutputStream unknown_fields_stream(
+ &unknown_fields_string);
+ // @@protoc_insertion_point(parse_start:pulsar.proto.CommandConsumerStatsResponse)
+ for (;;) {
+ ::std::pair< ::google::protobuf::uint32, bool> p = input->ReadTagWithCutoff(127);
+ tag = p.first;
+ if (!p.second) goto handle_unusual;
+ switch (::google::protobuf::internal::WireFormatLite::GetTagFieldNumber(tag)) {
+ // required uint64 request_id = 1;
+ case 1: {
+ if (tag == 8) {
+ DO_((::google::protobuf::internal::WireFormatLite::ReadPrimitive<
+ ::google::protobuf::uint64, ::google::protobuf::internal::WireFormatLite::TYPE_UINT64>(
+ input, &request_id_)));
+ set_has_request_id();
+ } else {
+ goto handle_unusual;
+ }
+ if (input->ExpectTag(16)) goto parse_error_code;
+ break;
+ }
+
+ // optional .pulsar.proto.ServerError error_code = 2;
+ case 2: {
+ if (tag == 16) {
+ parse_error_code:
+ int value;
+ DO_((::google::protobuf::internal::WireFormatLite::ReadPrimitive<
+ int, ::google::protobuf::internal::WireFormatLite::TYPE_ENUM>(
+ input, &value)));
+ if (::pulsar::proto::ServerError_IsValid(value)) {
+ set_error_code(static_cast< ::pulsar::proto::ServerError >(value));
+ } else {
+ unknown_fields_stream.WriteVarint32(tag);
+ unknown_fields_stream.WriteVarint32(value);
+ }
+ } else {
+ goto handle_unusual;
+ }
+ if (input->ExpectTag(26)) goto parse_error_message;
+ break;
+ }
+
+ // optional string error_message = 3;
+ case 3: {
+ if (tag == 26) {
+ parse_error_message:
+ DO_(::google::protobuf::internal::WireFormatLite::ReadString(
+ input, this->mutable_error_message()));
+ } else {
+ goto handle_unusual;
+ }
+ if (input->ExpectTag(33)) goto parse_msgRateOut;
+ break;
+ }
+
+ // optional double msgRateOut = 4;
+ case 4: {
+ if (tag == 33) {
+ parse_msgRateOut:
+ DO_((::google::protobuf::internal::WireFormatLite::ReadPrimitive<
+ double, ::google::protobuf::internal::WireFormatLite::TYPE_DOUBLE>(
+ input, &msgrateout_)));
+ set_has_msgrateout();
+ } else {
+ goto handle_unusual;
+ }
+ if (input->ExpectTag(41)) goto parse_msgThroughputOut;
+ break;
+ }
+
+ // optional double msgThroughputOut = 5;
+ case 5: {
+ if (tag == 41) {
+ parse_msgThroughputOut:
+ DO_((::google::protobuf::internal::WireFormatLite::ReadPrimitive<
+ double, ::google::protobuf::internal::WireFormatLite::TYPE_DOUBLE>(
+ input, &msgthroughputout_)));
+ set_has_msgthroughputout();
+ } else {
+ goto handle_unusual;
+ }
+ if (input->ExpectTag(49)) goto parse_msgRateRedeliver;
+ break;
+ }
+
+ // optional double msgRateRedeliver = 6;
+ case 6: {
+ if (tag == 49) {
+ parse_msgRateRedeliver:
+ DO_((::google::protobuf::internal::WireFormatLite::ReadPrimitive<
+ double, ::google::protobuf::internal::WireFormatLite::TYPE_DOUBLE>(
+ input, &msgrateredeliver_)));
+ set_has_msgrateredeliver();
+ } else {
+ goto handle_unusual;
+ }
+ if (input->ExpectTag(58)) goto parse_consumerName;
+ break;
+ }
+
+ // optional string consumerName = 7;
+ case 7: {
+ if (tag == 58) {
+ parse_consumerName:
+ DO_(::google::protobuf::internal::WireFormatLite::ReadString(
+ input, this->mutable_consumername()));
+ } else {
+ goto handle_unusual;
+ }
+ if (input->ExpectTag(64)) goto parse_availablePermits;
+ break;
+ }
+
+ // optional uint64 availablePermits = 8;
+ case 8: {
+ if (tag == 64) {
+ parse_availablePermits:
+ DO_((::google::protobuf::internal::WireFormatLite::ReadPrimitive<
+ ::google::protobuf::uint64, ::google::protobuf::internal::WireFormatLite::TYPE_UINT64>(
+ input, &availablepermits_)));
+ set_has_availablepermits();
+ } else {
+ goto handle_unusual;
+ }
+ if (input->ExpectTag(72)) goto parse_unackedMessages;
+ break;
+ }
+
+ // optional uint64 unackedMessages = 9;
+ case 9: {
+ if (tag == 72) {
+ parse_unackedMessages:
+ DO_((::google::protobuf::internal::WireFormatLite::ReadPrimitive<
+ ::google::protobuf::uint64, ::google::protobuf::internal::WireFormatLite::TYPE_UINT64>(
+ input, &unackedmessages_)));
+ set_has_unackedmessages();
+ } else {
+ goto handle_unusual;
+ }
+ if (input->ExpectTag(80)) goto parse_blockedConsumerOnUnackedMsgs;
+ break;
+ }
+
+ // optional bool blockedConsumerOnUnackedMsgs = 10;
+ case 10: {
+ if (tag == 80) {
+ parse_blockedConsumerOnUnackedMsgs:
+ DO_((::google::protobuf::internal::WireFormatLite::ReadPrimitive<
+ bool, ::google::protobuf::internal::WireFormatLite::TYPE_BOOL>(
+ input, &blockedconsumeronunackedmsgs_)));
+ set_has_blockedconsumeronunackedmsgs();
+ } else {
+ goto handle_unusual;
+ }
+ if (input->ExpectTag(90)) goto parse_address;
+ break;
+ }
+
+ // optional string address = 11;
+ case 11: {
+ if (tag == 90) {
+ parse_address:
+ DO_(::google::protobuf::internal::WireFormatLite::ReadString(
+ input, this->mutable_address()));
+ } else {
+ goto handle_unusual;
+ }
+ if (input->ExpectTag(98)) goto parse_connectedSince;
+ break;
+ }
+
+ // optional string connectedSince = 12;
+ case 12: {
+ if (tag == 98) {
+ parse_connectedSince:
+ DO_(::google::protobuf::internal::WireFormatLite::ReadString(
+ input, this->mutable_connectedsince()));
+ } else {
+ goto handle_unusual;
+ }
+ if (input->ExpectTag(106)) goto parse_type;
+ break;
+ }
+
+ // optional string type = 13;
+ case 13: {
+ if (tag == 106) {
+ parse_type:
+ DO_(::google::protobuf::internal::WireFormatLite::ReadString(
+ input, this->mutable_type()));
+ } else {
+ goto handle_unusual;
+ }
+ if (input->ExpectTag(113)) goto parse_msgRateExpired;
+ break;
+ }
+
+ // optional double msgRateExpired = 14;
+ case 14: {
+ if (tag == 113) {
+ parse_msgRateExpired:
+ DO_((::google::protobuf::internal::WireFormatLite::ReadPrimitive<
+ double, ::google::protobuf::internal::WireFormatLite::TYPE_DOUBLE>(
+ input, &msgrateexpired_)));
+ set_has_msgrateexpired();
+ } else {
+ goto handle_unusual;
+ }
+ if (input->ExpectTag(120)) goto parse_msgBacklog;
+ break;
+ }
+
+ // optional uint64 msgBacklog = 15;
+ case 15: {
+ if (tag == 120) {
+ parse_msgBacklog:
+ DO_((::google::protobuf::internal::WireFormatLite::ReadPrimitive<
+ ::google::protobuf::uint64, ::google::protobuf::internal::WireFormatLite::TYPE_UINT64>(
+ input, &msgbacklog_)));
+ set_has_msgbacklog();
+ } else {
+ goto handle_unusual;
+ }
+ if (input->ExpectAtEnd()) goto success;
+ break;
+ }
+
+ default: {
+ handle_unusual:
+ if (tag == 0 ||
+ ::google::protobuf::internal::WireFormatLite::GetTagWireType(tag) ==
+ ::google::protobuf::internal::WireFormatLite::WIRETYPE_END_GROUP) {
+ goto success;
+ }
+ DO_(::google::protobuf::internal::WireFormatLite::SkipField(
+ input, tag, &unknown_fields_stream));
+ break;
+ }
+ }
+ }
+success:
+ // @@protoc_insertion_point(parse_success:pulsar.proto.CommandConsumerStatsResponse)
+ return true;
+failure:
+ // @@protoc_insertion_point(parse_failure:pulsar.proto.CommandConsumerStatsResponse)
+ return false;
+#undef DO_
+}
+
+void CommandConsumerStatsResponse::SerializeWithCachedSizes(
+ ::google::protobuf::io::CodedOutputStream* output) const {
+ // @@protoc_insertion_point(serialize_start:pulsar.proto.CommandConsumerStatsResponse)
+ // required uint64 request_id = 1;
+ if (has_request_id()) {
+ ::google::protobuf::internal::WireFormatLite::WriteUInt64(1, this->request_id(), output);
+ }
+
+ // optional .pulsar.proto.ServerError error_code = 2;
+ if (has_error_code()) {
+ ::google::protobuf::internal::WireFormatLite::WriteEnum(
+ 2, this->error_code(), output);
+ }
+
+ // optional string error_message = 3;
+ if (has_error_message()) {
+ ::google::protobuf::internal::WireFormatLite::WriteStringMaybeAliased(
+ 3, this->error_message(), output);
+ }
+
+ // optional double msgRateOut = 4;
+ if (has_msgrateout()) {
+ ::google::protobuf::internal::WireFormatLite::WriteDouble(4, this->msgrateout(), output);
+ }
+
+ // optional double msgThroughputOut = 5;
+ if (has_msgthroughputout()) {
+ ::google::protobuf::internal::WireFormatLite::WriteDouble(5, this->msgthroughputout(), output);
+ }
+
+ // optional double msgRateRedeliver = 6;
+ if (has_msgrateredeliver()) {
+ ::google::protobuf::internal::WireFormatLite::WriteDouble(6, this->msgrateredeliver(), output);
+ }
+
+ // optional string consumerName = 7;
+ if (has_consumername()) {
+ ::google::protobuf::internal::WireFormatLite::WriteStringMaybeAliased(
+ 7, this->consumername(), output);
+ }
+
+ // optional uint64 availablePermits = 8;
+ if (has_availablepermits()) {
+ ::google::protobuf::internal::WireFormatLite::WriteUInt64(8, this->availablepermits(), output);
+ }
+
+ // optional uint64 unackedMessages = 9;
+ if (has_unackedmessages()) {
+ ::google::protobuf::internal::WireFormatLite::WriteUInt64(9, this->unackedmessages(), output);
+ }
+
+ // optional bool blockedConsumerOnUnackedMsgs = 10;
+ if (has_blockedconsumeronunackedmsgs()) {
+ ::google::protobuf::internal::WireFormatLite::WriteBool(10, this->blockedconsumeronunackedmsgs(), output);
+ }
+
+ // optional string address = 11;
+ if (has_address()) {
+ ::google::protobuf::internal::WireFormatLite::WriteStringMaybeAliased(
+ 11, this->address(), output);
+ }
+
+ // optional string connectedSince = 12;
+ if (has_connectedsince()) {
+ ::google::protobuf::internal::WireFormatLite::WriteStringMaybeAliased(
+ 12, this->connectedsince(), output);
+ }
+
+ // optional string type = 13;
+ if (has_type()) {
+ ::google::protobuf::internal::WireFormatLite::WriteStringMaybeAliased(
+ 13, this->type(), output);
+ }
+
+ // optional double msgRateExpired = 14;
+ if (has_msgrateexpired()) {
+ ::google::protobuf::internal::WireFormatLite::WriteDouble(14, this->msgrateexpired(), output);
+ }
+
+ // optional uint64 msgBacklog = 15;
+ if (has_msgbacklog()) {
+ ::google::protobuf::internal::WireFormatLite::WriteUInt64(15, this->msgbacklog(), output);
+ }
+
+ output->WriteRaw(unknown_fields().data(),
+ unknown_fields().size());
+ // @@protoc_insertion_point(serialize_end:pulsar.proto.CommandConsumerStatsResponse)
+}
+
+int CommandConsumerStatsResponse::ByteSize() const {
+ int total_size = 0;
+
+ if (_has_bits_[0 / 32] & (0xffu << (0 % 32))) {
+ // required uint64 request_id = 1;
+ if (has_request_id()) {
+ total_size += 1 +
+ ::google::protobuf::internal::WireFormatLite::UInt64Size(
+ this->request_id());
+ }
+
+ // optional .pulsar.proto.ServerError error_code = 2;
+ if (has_error_code()) {
+ total_size += 1 +
+ ::google::protobuf::internal::WireFormatLite::EnumSize(this->error_code());
+ }
+
+ // optional string error_message = 3;
+ if (has_error_message()) {
+ total_size += 1 +
+ ::google::protobuf::internal::WireFormatLite::StringSize(
+ this->error_message());
+ }
+
+ // optional double msgRateOut = 4;
+ if (has_msgrateout()) {
+ total_size += 1 + 8;
+ }
+
+ // optional double msgThroughputOut = 5;
+ if (has_msgthroughputout()) {
+ total_size += 1 + 8;
+ }
+
+ // optional double msgRateRedeliver = 6;
+ if (has_msgrateredeliver()) {
+ total_size += 1 + 8;
+ }
+
+ // optional string consumerName = 7;
+ if (has_consumername()) {
+ total_size += 1 +
+ ::google::protobuf::internal::WireFormatLite::StringSize(
+ this->consumername());
+ }
+
+ // optional uint64 availablePermits = 8;
+ if (has_availablepermits()) {
+ total_size += 1 +
+ ::google::protobuf::internal::WireFormatLite::UInt64Size(
+ this->availablepermits());
+ }
+
+ }
+ if (_has_bits_[8 / 32] & (0xffu << (8 % 32))) {
+ // optional uint64 unackedMessages = 9;
+ if (has_unackedmessages()) {
+ total_size += 1 +
+ ::google::protobuf::internal::WireFormatLite::UInt64Size(
+ this->unackedmessages());
+ }
+
+ // optional bool blockedConsumerOnUnackedMsgs = 10;
+ if (has_blockedconsumeronunackedmsgs()) {
+ total_size += 1 + 1;
+ }
+
+ // optional string address = 11;
+ if (has_address()) {
+ total_size += 1 +
+ ::google::protobuf::internal::WireFormatLite::StringSize(
+ this->address());
+ }
+
+ // optional string connectedSince = 12;
+ if (has_connectedsince()) {
+ total_size += 1 +
+ ::google::protobuf::internal::WireFormatLite::StringSize(
+ this->connectedsince());
+ }
+
+ // optional string type = 13;
+ if (has_type()) {
+ total_size += 1 +
+ ::google::protobuf::internal::WireFormatLite::StringSize(
+ this->type());
+ }
+
+ // optional double msgRateExpired = 14;
+ if (has_msgrateexpired()) {
+ total_size += 1 + 8;
+ }
+
+ // optional uint64 msgBacklog = 15;
+ if (has_msgbacklog()) {
+ total_size += 1 +
+ ::google::protobuf::internal::WireFormatLite::UInt64Size(
+ this->msgbacklog());
+ }
+
+ }
+ total_size += unknown_fields().size();
+
+ GOOGLE_SAFE_CONCURRENT_WRITES_BEGIN();
+ _cached_size_ = total_size;
+ GOOGLE_SAFE_CONCURRENT_WRITES_END();
+ return total_size;
+}
+
+void CommandConsumerStatsResponse::CheckTypeAndMergeFrom(
+ const ::google::protobuf::MessageLite& from) {
+ MergeFrom(*::google::protobuf::down_cast(&from));
+}
+
+void CommandConsumerStatsResponse::MergeFrom(const CommandConsumerStatsResponse& from) {
+ GOOGLE_CHECK_NE(&from, this);
+ if (from._has_bits_[0 / 32] & (0xffu << (0 % 32))) {
+ if (from.has_request_id()) {
+ set_request_id(from.request_id());
+ }
+ if (from.has_error_code()) {
+ set_error_code(from.error_code());
+ }
+ if (from.has_error_message()) {
+ set_error_message(from.error_message());
+ }
+ if (from.has_msgrateout()) {
+ set_msgrateout(from.msgrateout());
+ }
+ if (from.has_msgthroughputout()) {
+ set_msgthroughputout(from.msgthroughputout());
+ }
+ if (from.has_msgrateredeliver()) {
+ set_msgrateredeliver(from.msgrateredeliver());
+ }
+ if (from.has_consumername()) {
+ set_consumername(from.consumername());
+ }
+ if (from.has_availablepermits()) {
+ set_availablepermits(from.availablepermits());
+ }
+ }
+ if (from._has_bits_[8 / 32] & (0xffu << (8 % 32))) {
+ if (from.has_unackedmessages()) {
+ set_unackedmessages(from.unackedmessages());
+ }
+ if (from.has_blockedconsumeronunackedmsgs()) {
+ set_blockedconsumeronunackedmsgs(from.blockedconsumeronunackedmsgs());
+ }
+ if (from.has_address()) {
+ set_address(from.address());
+ }
+ if (from.has_connectedsince()) {
+ set_connectedsince(from.connectedsince());
+ }
+ if (from.has_type()) {
+ set_type(from.type());
+ }
+ if (from.has_msgrateexpired()) {
+ set_msgrateexpired(from.msgrateexpired());
+ }
+ if (from.has_msgbacklog()) {
+ set_msgbacklog(from.msgbacklog());
+ }
+ }
+ mutable_unknown_fields()->append(from.unknown_fields());
+}
+
+void CommandConsumerStatsResponse::CopyFrom(const CommandConsumerStatsResponse& from) {
+ if (&from == this) return;
+ Clear();
+ MergeFrom(from);
+}
+
+bool CommandConsumerStatsResponse::IsInitialized() const {
+ if ((_has_bits_[0] & 0x00000001) != 0x00000001) return false;
+
+ return true;
+}
+
+void CommandConsumerStatsResponse::Swap(CommandConsumerStatsResponse* other) {
+ if (other != this) {
+ std::swap(request_id_, other->request_id_);
+ std::swap(error_code_, other->error_code_);
+ std::swap(error_message_, other->error_message_);
+ std::swap(msgrateout_, other->msgrateout_);
+ std::swap(msgthroughputout_, other->msgthroughputout_);
+ std::swap(msgrateredeliver_, other->msgrateredeliver_);
+ std::swap(consumername_, other->consumername_);
+ std::swap(availablepermits_, other->availablepermits_);
+ std::swap(unackedmessages_, other->unackedmessages_);
+ std::swap(blockedconsumeronunackedmsgs_, other->blockedconsumeronunackedmsgs_);
+ std::swap(address_, other->address_);
+ std::swap(connectedsince_, other->connectedsince_);
+ std::swap(type_, other->type_);
+ std::swap(msgrateexpired_, other->msgrateexpired_);
+ std::swap(msgbacklog_, other->msgbacklog_);
+ std::swap(_has_bits_[0], other->_has_bits_[0]);
+ _unknown_fields_.swap(other->_unknown_fields_);
+ std::swap(_cached_size_, other->_cached_size_);
+ }
+}
+
+::std::string CommandConsumerStatsResponse::GetTypeName() const {
+ return "pulsar.proto.CommandConsumerStatsResponse";
+}
+
+
+// ===================================================================
+
+bool BaseCommand_Type_IsValid(int value) {
+ switch(value) {
+ case 2:
+ case 3:
+ case 4:
+ case 5:
+ case 6:
+ case 7:
+ case 8:
+ case 9:
+ case 10:
+ case 11:
+ case 12:
+ case 13:
+ case 14:
+ case 15:
+ case 16:
+ case 17:
+ case 18:
+ case 19:
+ case 20:
+ case 21:
+ case 22:
+ case 23:
+ case 24:
+ case 25:
+ case 26:
+ return true;
+ default:
+ return false;
+ }
+}
+
+#ifndef _MSC_VER
+const BaseCommand_Type BaseCommand::CONNECT;
+const BaseCommand_Type BaseCommand::CONNECTED;
+const BaseCommand_Type BaseCommand::SUBSCRIBE;
+const BaseCommand_Type BaseCommand::PRODUCER;
+const BaseCommand_Type BaseCommand::SEND;
+const BaseCommand_Type BaseCommand::SEND_RECEIPT;
+const BaseCommand_Type BaseCommand::SEND_ERROR;
+const BaseCommand_Type BaseCommand::MESSAGE;
+const BaseCommand_Type BaseCommand::ACK;
+const BaseCommand_Type BaseCommand::FLOW;
+const BaseCommand_Type BaseCommand::UNSUBSCRIBE;
+const BaseCommand_Type BaseCommand::SUCCESS;
+const BaseCommand_Type BaseCommand::ERROR;
+const BaseCommand_Type BaseCommand::CLOSE_PRODUCER;
+const BaseCommand_Type BaseCommand::CLOSE_CONSUMER;
+const BaseCommand_Type BaseCommand::PRODUCER_SUCCESS;
+const BaseCommand_Type BaseCommand::PING;
+const BaseCommand_Type BaseCommand::PONG;
+const BaseCommand_Type BaseCommand::REDELIVER_UNACKNOWLEDGED_MESSAGES;
+const BaseCommand_Type BaseCommand::PARTITIONED_METADATA;
+const BaseCommand_Type BaseCommand::PARTITIONED_METADATA_RESPONSE;
+const BaseCommand_Type BaseCommand::LOOKUP;
+const BaseCommand_Type BaseCommand::LOOKUP_RESPONSE;
+const BaseCommand_Type BaseCommand::CONSUMER_STATS;
+const BaseCommand_Type BaseCommand::CONSUMER_STATS_RESPONSE;
+const BaseCommand_Type BaseCommand::Type_MIN;
+const BaseCommand_Type BaseCommand::Type_MAX;
+const int BaseCommand::Type_ARRAYSIZE;
+#endif // _MSC_VER
+#ifndef _MSC_VER
+const int BaseCommand::kTypeFieldNumber;
+const int BaseCommand::kConnectFieldNumber;
+const int BaseCommand::kConnectedFieldNumber;
+const int BaseCommand::kSubscribeFieldNumber;
+const int BaseCommand::kProducerFieldNumber;
+const int BaseCommand::kSendFieldNumber;
+const int BaseCommand::kSendReceiptFieldNumber;
+const int BaseCommand::kSendErrorFieldNumber;
+const int BaseCommand::kMessageFieldNumber;
+const int BaseCommand::kAckFieldNumber;
+const int BaseCommand::kFlowFieldNumber;
+const int BaseCommand::kUnsubscribeFieldNumber;
+const int BaseCommand::kSuccessFieldNumber;
+const int BaseCommand::kErrorFieldNumber;
+const int BaseCommand::kCloseProducerFieldNumber;
+const int BaseCommand::kCloseConsumerFieldNumber;
+const int BaseCommand::kProducerSuccessFieldNumber;
+const int BaseCommand::kPingFieldNumber;
+const int BaseCommand::kPongFieldNumber;
+const int BaseCommand::kRedeliverUnacknowledgedMessagesFieldNumber;
+const int BaseCommand::kPartitionMetadataFieldNumber;
+const int BaseCommand::kPartitionMetadataResponseFieldNumber;
+const int BaseCommand::kLookupTopicFieldNumber;
+const int BaseCommand::kLookupTopicResponseFieldNumber;
+const int BaseCommand::kConsumerStatsFieldNumber;
+const int BaseCommand::kConsumerStatsResponseFieldNumber;
+#endif // !_MSC_VER
+
+BaseCommand::BaseCommand()
+ : ::google::protobuf::MessageLite() {
+ SharedCtor();
+ // @@protoc_insertion_point(constructor:pulsar.proto.BaseCommand)
+}
+
+void BaseCommand::InitAsDefaultInstance() {
+#ifdef GOOGLE_PROTOBUF_NO_STATIC_INITIALIZER
+ connect_ = const_cast< ::pulsar::proto::CommandConnect*>(
+ ::pulsar::proto::CommandConnect::internal_default_instance());
+#else
+ connect_ = const_cast< ::pulsar::proto::CommandConnect*>(&::pulsar::proto::CommandConnect::default_instance());
+#endif
+#ifdef GOOGLE_PROTOBUF_NO_STATIC_INITIALIZER
+ connected_ = const_cast< ::pulsar::proto::CommandConnected*>(
+ ::pulsar::proto::CommandConnected::internal_default_instance());
+#else
+ connected_ = const_cast< ::pulsar::proto::CommandConnected*>(&::pulsar::proto::CommandConnected::default_instance());
+#endif
+#ifdef GOOGLE_PROTOBUF_NO_STATIC_INITIALIZER
+ subscribe_ = const_cast< ::pulsar::proto::CommandSubscribe*>(
+ ::pulsar::proto::CommandSubscribe::internal_default_instance());
+#else
+ subscribe_ = const_cast< ::pulsar::proto::CommandSubscribe*>(&::pulsar::proto::CommandSubscribe::default_instance());
+#endif
+#ifdef GOOGLE_PROTOBUF_NO_STATIC_INITIALIZER
+ producer_ = const_cast< ::pulsar::proto::CommandProducer*>(
+ ::pulsar::proto::CommandProducer::internal_default_instance());
+#else
+ producer_ = const_cast< ::pulsar::proto::CommandProducer*>(&::pulsar::proto::CommandProducer::default_instance());
+#endif
+#ifdef GOOGLE_PROTOBUF_NO_STATIC_INITIALIZER
+ send_ = const_cast< ::pulsar::proto::CommandSend*>(
+ ::pulsar::proto::CommandSend::internal_default_instance());
+#else
+ send_ = const_cast< ::pulsar::proto::CommandSend*>(&::pulsar::proto::CommandSend::default_instance());
+#endif
+#ifdef GOOGLE_PROTOBUF_NO_STATIC_INITIALIZER
+ send_receipt_ = const_cast< ::pulsar::proto::CommandSendReceipt*>(
+ ::pulsar::proto::CommandSendReceipt::internal_default_instance());
+#else
+ send_receipt_ = const_cast< ::pulsar::proto::CommandSendReceipt*>(&::pulsar::proto::CommandSendReceipt::default_instance());
+#endif
+#ifdef GOOGLE_PROTOBUF_NO_STATIC_INITIALIZER
+ send_error_ = const_cast< ::pulsar::proto::CommandSendError*>(
+ ::pulsar::proto::CommandSendError::internal_default_instance());
+#else
+ send_error_ = const_cast< ::pulsar::proto::CommandSendError*>(&::pulsar::proto::CommandSendError::default_instance());
+#endif
+#ifdef GOOGLE_PROTOBUF_NO_STATIC_INITIALIZER
+ message_ = const_cast< ::pulsar::proto::CommandMessage*>(
+ ::pulsar::proto::CommandMessage::internal_default_instance());
+#else
+ message_ = const_cast< ::pulsar::proto::CommandMessage*>(&::pulsar::proto::CommandMessage::default_instance());
+#endif
+#ifdef GOOGLE_PROTOBUF_NO_STATIC_INITIALIZER
+ ack_ = const_cast< ::pulsar::proto::CommandAck*>(
+ ::pulsar::proto::CommandAck::internal_default_instance());
+#else
+ ack_ = const_cast< ::pulsar::proto::CommandAck*>(&::pulsar::proto::CommandAck::default_instance());
+#endif
+#ifdef GOOGLE_PROTOBUF_NO_STATIC_INITIALIZER
+ flow_ = const_cast< ::pulsar::proto::CommandFlow*>(
+ ::pulsar::proto::CommandFlow::internal_default_instance());
+#else
+ flow_ = const_cast< ::pulsar::proto::CommandFlow*>(&::pulsar::proto::CommandFlow::default_instance());
+#endif
+#ifdef GOOGLE_PROTOBUF_NO_STATIC_INITIALIZER
+ unsubscribe_ = const_cast< ::pulsar::proto::CommandUnsubscribe*>(
+ ::pulsar::proto::CommandUnsubscribe::internal_default_instance());
+#else
+ unsubscribe_ = const_cast< ::pulsar::proto::CommandUnsubscribe*>(&::pulsar::proto::CommandUnsubscribe::default_instance());
+#endif
+#ifdef GOOGLE_PROTOBUF_NO_STATIC_INITIALIZER
+ success_ = const_cast< ::pulsar::proto::CommandSuccess*>(
+ ::pulsar::proto::CommandSuccess::internal_default_instance());
+#else
+ success_ = const_cast< ::pulsar::proto::CommandSuccess*>(&::pulsar::proto::CommandSuccess::default_instance());
+#endif
+#ifdef GOOGLE_PROTOBUF_NO_STATIC_INITIALIZER
+ error_ = const_cast< ::pulsar::proto::CommandError*>(
+ ::pulsar::proto::CommandError::internal_default_instance());
+#else
+ error_ = const_cast< ::pulsar::proto::CommandError*>(&::pulsar::proto::CommandError::default_instance());
+#endif
+#ifdef GOOGLE_PROTOBUF_NO_STATIC_INITIALIZER
+ close_producer_ = const_cast< ::pulsar::proto::CommandCloseProducer*>(
+ ::pulsar::proto::CommandCloseProducer::internal_default_instance());
+#else
+ close_producer_ = const_cast< ::pulsar::proto::CommandCloseProducer*>(&::pulsar::proto::CommandCloseProducer::default_instance());
+#endif
+#ifdef GOOGLE_PROTOBUF_NO_STATIC_INITIALIZER
+ close_consumer_ = const_cast< ::pulsar::proto::CommandCloseConsumer*>(
+ ::pulsar::proto::CommandCloseConsumer::internal_default_instance());
+#else
+ close_consumer_ = const_cast< ::pulsar::proto::CommandCloseConsumer*>(&::pulsar::proto::CommandCloseConsumer::default_instance());
+#endif
+#ifdef GOOGLE_PROTOBUF_NO_STATIC_INITIALIZER
+ producer_success_ = const_cast< ::pulsar::proto::CommandProducerSuccess*>(
+ ::pulsar::proto::CommandProducerSuccess::internal_default_instance());
+#else
+ producer_success_ = const_cast< ::pulsar::proto::CommandProducerSuccess*>(&::pulsar::proto::CommandProducerSuccess::default_instance());
+#endif
+#ifdef GOOGLE_PROTOBUF_NO_STATIC_INITIALIZER
+ ping_ = const_cast< ::pulsar::proto::CommandPing*>(
+ ::pulsar::proto::CommandPing::internal_default_instance());
+#else
+ ping_ = const_cast< ::pulsar::proto::CommandPing*>(&::pulsar::proto::CommandPing::default_instance());
+#endif
+#ifdef GOOGLE_PROTOBUF_NO_STATIC_INITIALIZER
+ pong_ = const_cast< ::pulsar::proto::CommandPong*>(
+ ::pulsar::proto::CommandPong::internal_default_instance());
+#else
+ pong_ = const_cast< ::pulsar::proto::CommandPong*>(&::pulsar::proto::CommandPong::default_instance());
+#endif
+#ifdef GOOGLE_PROTOBUF_NO_STATIC_INITIALIZER
+ redeliverunacknowledgedmessages_ = const_cast< ::pulsar::proto::CommandRedeliverUnacknowledgedMessages*>(
+ ::pulsar::proto::CommandRedeliverUnacknowledgedMessages::internal_default_instance());
+#else
+ redeliverunacknowledgedmessages_ = const_cast< ::pulsar::proto::CommandRedeliverUnacknowledgedMessages*>(&::pulsar::proto::CommandRedeliverUnacknowledgedMessages::default_instance());
+#endif
+#ifdef GOOGLE_PROTOBUF_NO_STATIC_INITIALIZER
+ partitionmetadata_ = const_cast< ::pulsar::proto::CommandPartitionedTopicMetadata*>(
+ ::pulsar::proto::CommandPartitionedTopicMetadata::internal_default_instance());
+#else
+ partitionmetadata_ = const_cast< ::pulsar::proto::CommandPartitionedTopicMetadata*>(&::pulsar::proto::CommandPartitionedTopicMetadata::default_instance());
+#endif
+#ifdef GOOGLE_PROTOBUF_NO_STATIC_INITIALIZER
+ partitionmetadataresponse_ = const_cast< ::pulsar::proto::CommandPartitionedTopicMetadataResponse*>(
+ ::pulsar::proto::CommandPartitionedTopicMetadataResponse::internal_default_instance());
+#else
+ partitionmetadataresponse_ = const_cast< ::pulsar::proto::CommandPartitionedTopicMetadataResponse*>(&::pulsar::proto::CommandPartitionedTopicMetadataResponse::default_instance());
+#endif
+#ifdef GOOGLE_PROTOBUF_NO_STATIC_INITIALIZER
+ lookuptopic_ = const_cast< ::pulsar::proto::CommandLookupTopic*>(
+ ::pulsar::proto::CommandLookupTopic::internal_default_instance());
+#else
+ lookuptopic_ = const_cast< ::pulsar::proto::CommandLookupTopic*>(&::pulsar::proto::CommandLookupTopic::default_instance());
+#endif
+#ifdef GOOGLE_PROTOBUF_NO_STATIC_INITIALIZER
+ lookuptopicresponse_ = const_cast< ::pulsar::proto::CommandLookupTopicResponse*>(
+ ::pulsar::proto::CommandLookupTopicResponse::internal_default_instance());
+#else
+ lookuptopicresponse_ = const_cast< ::pulsar::proto::CommandLookupTopicResponse*>(&::pulsar::proto::CommandLookupTopicResponse::default_instance());
+#endif
+#ifdef GOOGLE_PROTOBUF_NO_STATIC_INITIALIZER
+ consumerstats_ = const_cast< ::pulsar::proto::CommandConsumerStats*>(
+ ::pulsar::proto::CommandConsumerStats::internal_default_instance());
+#else
+ consumerstats_ = const_cast< ::pulsar::proto::CommandConsumerStats*>(&::pulsar::proto::CommandConsumerStats::default_instance());
+#endif
+#ifdef GOOGLE_PROTOBUF_NO_STATIC_INITIALIZER
+ consumerstatsresponse_ = const_cast< ::pulsar::proto::CommandConsumerStatsResponse*>(
+ ::pulsar::proto::CommandConsumerStatsResponse::internal_default_instance());
+#else
+ consumerstatsresponse_ = const_cast< ::pulsar::proto::CommandConsumerStatsResponse*>(&::pulsar::proto::CommandConsumerStatsResponse::default_instance());
+#endif
+}
+
+BaseCommand::BaseCommand(const BaseCommand& from)
+ : ::google::protobuf::MessageLite() {
+ SharedCtor();
+ MergeFrom(from);
+ // @@protoc_insertion_point(copy_constructor:pulsar.proto.BaseCommand)
+}
+
+void BaseCommand::SharedCtor() {
+ _cached_size_ = 0;
+ type_ = 2;
+ connect_ = NULL;
+ connected_ = NULL;
+ subscribe_ = NULL;
+ producer_ = NULL;
+ send_ = NULL;
+ send_receipt_ = NULL;
+ send_error_ = NULL;
+ message_ = NULL;
+ ack_ = NULL;
+ flow_ = NULL;
+ unsubscribe_ = NULL;
+ success_ = NULL;
+ error_ = NULL;
+ close_producer_ = NULL;
+ close_consumer_ = NULL;
+ producer_success_ = NULL;
+ ping_ = NULL;
+ pong_ = NULL;
+ redeliverunacknowledgedmessages_ = NULL;
+ partitionmetadata_ = NULL;
+ partitionmetadataresponse_ = NULL;
+ lookuptopic_ = NULL;
lookuptopicresponse_ = NULL;
+ consumerstats_ = NULL;
+ consumerstatsresponse_ = NULL;
::memset(_has_bits_, 0, sizeof(_has_bits_));
}
@@ -8220,6 +9273,8 @@ void BaseCommand::SharedDtor() {
delete partitionmetadataresponse_;
delete lookuptopic_;
delete lookuptopicresponse_;
+ delete consumerstats_;
+ delete consumerstatsresponse_;
}
}
@@ -8320,6 +9375,14 @@ void BaseCommand::Clear() {
if (lookuptopicresponse_ != NULL) lookuptopicresponse_->::pulsar::proto::CommandLookupTopicResponse::Clear();
}
}
+ if (_has_bits_[24 / 32] & 50331648) {
+ if (has_consumerstats()) {
+ if (consumerstats_ != NULL) consumerstats_->::pulsar::proto::CommandConsumerStats::Clear();
+ }
+ if (has_consumerstatsresponse()) {
+ if (consumerstatsresponse_ != NULL) consumerstatsresponse_->::pulsar::proto::CommandConsumerStatsResponse::Clear();
+ }
+ }
::memset(_has_bits_, 0, sizeof(_has_bits_));
mutable_unknown_fields()->clear();
}
@@ -8653,6 +9716,32 @@ bool BaseCommand::MergePartialFromCodedStream(
} else {
goto handle_unusual;
}
+ if (input->ExpectTag(202)) goto parse_consumerStats;
+ break;
+ }
+
+ // optional .pulsar.proto.CommandConsumerStats consumerStats = 25;
+ case 25: {
+ if (tag == 202) {
+ parse_consumerStats:
+ DO_(::google::protobuf::internal::WireFormatLite::ReadMessageNoVirtual(
+ input, mutable_consumerstats()));
+ } else {
+ goto handle_unusual;
+ }
+ if (input->ExpectTag(210)) goto parse_consumerStatsResponse;
+ break;
+ }
+
+ // optional .pulsar.proto.CommandConsumerStatsResponse consumerStatsResponse = 26;
+ case 26: {
+ if (tag == 210) {
+ parse_consumerStatsResponse:
+ DO_(::google::protobuf::internal::WireFormatLite::ReadMessageNoVirtual(
+ input, mutable_consumerstatsresponse()));
+ } else {
+ goto handle_unusual;
+ }
if (input->ExpectAtEnd()) goto success;
break;
}
@@ -8826,6 +9915,18 @@ void BaseCommand::SerializeWithCachedSizes(
24, this->lookuptopicresponse(), output);
}
+ // optional .pulsar.proto.CommandConsumerStats consumerStats = 25;
+ if (has_consumerstats()) {
+ ::google::protobuf::internal::WireFormatLite::WriteMessage(
+ 25, this->consumerstats(), output);
+ }
+
+ // optional .pulsar.proto.CommandConsumerStatsResponse consumerStatsResponse = 26;
+ if (has_consumerstatsresponse()) {
+ ::google::protobuf::internal::WireFormatLite::WriteMessage(
+ 26, this->consumerstatsresponse(), output);
+ }
+
output->WriteRaw(unknown_fields().data(),
unknown_fields().size());
// @@protoc_insertion_point(serialize_end:pulsar.proto.BaseCommand)
@@ -9006,6 +10107,22 @@ int BaseCommand::ByteSize() const {
this->lookuptopicresponse());
}
+ }
+ if (_has_bits_[24 / 32] & (0xffu << (24 % 32))) {
+ // optional .pulsar.proto.CommandConsumerStats consumerStats = 25;
+ if (has_consumerstats()) {
+ total_size += 2 +
+ ::google::protobuf::internal::WireFormatLite::MessageSizeNoVirtual(
+ this->consumerstats());
+ }
+
+ // optional .pulsar.proto.CommandConsumerStatsResponse consumerStatsResponse = 26;
+ if (has_consumerstatsresponse()) {
+ total_size += 2 +
+ ::google::protobuf::internal::WireFormatLite::MessageSizeNoVirtual(
+ this->consumerstatsresponse());
+ }
+
}
total_size += unknown_fields().size();
@@ -9100,6 +10217,14 @@ void BaseCommand::MergeFrom(const BaseCommand& from) {
mutable_lookuptopicresponse()->::pulsar::proto::CommandLookupTopicResponse::MergeFrom(from.lookuptopicresponse());
}
}
+ if (from._has_bits_[24 / 32] & (0xffu << (24 % 32))) {
+ if (from.has_consumerstats()) {
+ mutable_consumerstats()->::pulsar::proto::CommandConsumerStats::MergeFrom(from.consumerstats());
+ }
+ if (from.has_consumerstatsresponse()) {
+ mutable_consumerstatsresponse()->::pulsar::proto::CommandConsumerStatsResponse::MergeFrom(from.consumerstatsresponse());
+ }
+ }
mutable_unknown_fields()->append(from.unknown_fields());
}
@@ -9175,6 +10300,12 @@ bool BaseCommand::IsInitialized() const {
if (has_lookuptopicresponse()) {
if (!this->lookuptopicresponse().IsInitialized()) return false;
}
+ if (has_consumerstats()) {
+ if (!this->consumerstats().IsInitialized()) return false;
+ }
+ if (has_consumerstatsresponse()) {
+ if (!this->consumerstatsresponse().IsInitialized()) return false;
+ }
return true;
}
@@ -9204,6 +10335,8 @@ void BaseCommand::Swap(BaseCommand* other) {
std::swap(partitionmetadataresponse_, other->partitionmetadataresponse_);
std::swap(lookuptopic_, other->lookuptopic_);
std::swap(lookuptopicresponse_, other->lookuptopicresponse_);
+ std::swap(consumerstats_, other->consumerstats_);
+ std::swap(consumerstatsresponse_, other->consumerstatsresponse_);
std::swap(_has_bits_[0], other->_has_bits_[0]);
_unknown_fields_.swap(other->_unknown_fields_);
std::swap(_cached_size_, other->_cached_size_);
diff --git a/pulsar-client-cpp/lib/PulsarApi.pb.h b/pulsar-client-cpp/lib/PulsarApi.pb.h
index df8733f5ea380..91e973baeb169 100644
--- a/pulsar-client-cpp/lib/PulsarApi.pb.h
+++ b/pulsar-client-cpp/lib/PulsarApi.pb.h
@@ -13,7 +13,7 @@
#error incompatible with your Protocol Buffer headers. Please update
#error your headers.
#endif
-#if 2006000 < GOOGLE_PROTOBUF_MIN_PROTOC_VERSION
+#if 2006001 < GOOGLE_PROTOBUF_MIN_PROTOC_VERSION
#error This file was generated by an older version of protoc which is
#error incompatible with your Protocol Buffer headers. Please
#error regenerate this file with a newer version of protoc.
@@ -60,6 +60,8 @@ class CommandProducerSuccess;
class CommandError;
class CommandPing;
class CommandPong;
+class CommandConsumerStats;
+class CommandConsumerStatsResponse;
class BaseCommand;
enum CommandSubscribe_SubType {
@@ -134,11 +136,13 @@ enum BaseCommand_Type {
BaseCommand_Type_PARTITIONED_METADATA = 21,
BaseCommand_Type_PARTITIONED_METADATA_RESPONSE = 22,
BaseCommand_Type_LOOKUP = 23,
- BaseCommand_Type_LOOKUP_RESPONSE = 24
+ BaseCommand_Type_LOOKUP_RESPONSE = 24,
+ BaseCommand_Type_CONSUMER_STATS = 25,
+ BaseCommand_Type_CONSUMER_STATS_RESPONSE = 26
};
bool BaseCommand_Type_IsValid(int value);
const BaseCommand_Type BaseCommand_Type_Type_MIN = BaseCommand_Type_CONNECT;
-const BaseCommand_Type BaseCommand_Type_Type_MAX = BaseCommand_Type_LOOKUP_RESPONSE;
+const BaseCommand_Type BaseCommand_Type_Type_MAX = BaseCommand_Type_CONSUMER_STATS_RESPONSE;
const int BaseCommand_Type_Type_ARRAYSIZE = BaseCommand_Type_Type_MAX + 1;
enum CompressionType {
@@ -161,11 +165,15 @@ enum ServerError {
ServiceNotReady = 6,
ProducerBlockedQuotaExceededError = 7,
ProducerBlockedQuotaExceededException = 8,
- ChecksumError = 9
+ ChecksumError = 9,
+ UnsupportedVersionError = 10,
+ TopicNotFound = 11,
+ SubscriptionNotFound = 12,
+ ConsumerNotFound = 13
};
bool ServerError_IsValid(int value);
const ServerError ServerError_MIN = UnknownError;
-const ServerError ServerError_MAX = ChecksumError;
+const ServerError ServerError_MAX = ConsumerNotFound;
const int ServerError_ARRAYSIZE = ServerError_MAX + 1;
enum AuthMethod {
@@ -185,11 +193,12 @@ enum ProtocolVersion {
v3 = 3,
v4 = 4,
v5 = 5,
- v6 = 6
+ v6 = 6,
+ v7 = 7
};
bool ProtocolVersion_IsValid(int value);
const ProtocolVersion ProtocolVersion_MIN = v0;
-const ProtocolVersion ProtocolVersion_MAX = v6;
+const ProtocolVersion ProtocolVersion_MAX = v7;
const int ProtocolVersion_ARRAYSIZE = ProtocolVersion_MAX + 1;
// ===================================================================
@@ -3442,6 +3451,393 @@ class CommandPong : public ::google::protobuf::MessageLite {
};
// -------------------------------------------------------------------
+class CommandConsumerStats : public ::google::protobuf::MessageLite {
+ public:
+ CommandConsumerStats();
+ virtual ~CommandConsumerStats();
+
+ CommandConsumerStats(const CommandConsumerStats& from);
+
+ inline CommandConsumerStats& operator=(const CommandConsumerStats& from) {
+ CopyFrom(from);
+ return *this;
+ }
+
+ inline const ::std::string& unknown_fields() const {
+ return _unknown_fields_;
+ }
+
+ inline ::std::string* mutable_unknown_fields() {
+ return &_unknown_fields_;
+ }
+
+ static const CommandConsumerStats& default_instance();
+
+ #ifdef GOOGLE_PROTOBUF_NO_STATIC_INITIALIZER
+ // Returns the internal default instance pointer. This function can
+ // return NULL thus should not be used by the user. This is intended
+ // for Protobuf internal code. Please use default_instance() declared
+ // above instead.
+ static inline const CommandConsumerStats* internal_default_instance() {
+ return default_instance_;
+ }
+ #endif
+
+ void Swap(CommandConsumerStats* other);
+
+ // implements Message ----------------------------------------------
+
+ CommandConsumerStats* New() const;
+ void CheckTypeAndMergeFrom(const ::google::protobuf::MessageLite& from);
+ void CopyFrom(const CommandConsumerStats& from);
+ void MergeFrom(const CommandConsumerStats& from);
+ void Clear();
+ bool IsInitialized() const;
+
+ int ByteSize() const;
+ bool MergePartialFromCodedStream(
+ ::google::protobuf::io::CodedInputStream* input);
+ void SerializeWithCachedSizes(
+ ::google::protobuf::io::CodedOutputStream* output) const;
+ void DiscardUnknownFields();
+ int GetCachedSize() const { return _cached_size_; }
+ private:
+ void SharedCtor();
+ void SharedDtor();
+ void SetCachedSize(int size) const;
+ public:
+ ::std::string GetTypeName() const;
+
+ // nested types ----------------------------------------------------
+
+ // accessors -------------------------------------------------------
+
+ // required uint64 request_id = 1;
+ inline bool has_request_id() const;
+ inline void clear_request_id();
+ static const int kRequestIdFieldNumber = 1;
+ inline ::google::protobuf::uint64 request_id() const;
+ inline void set_request_id(::google::protobuf::uint64 value);
+
+ // required string topic_name = 2;
+ inline bool has_topic_name() const;
+ inline void clear_topic_name();
+ static const int kTopicNameFieldNumber = 2;
+ inline const ::std::string& topic_name() const;
+ inline void set_topic_name(const ::std::string& value);
+ inline void set_topic_name(const char* value);
+ inline void set_topic_name(const char* value, size_t size);
+ inline ::std::string* mutable_topic_name();
+ inline ::std::string* release_topic_name();
+ inline void set_allocated_topic_name(::std::string* topic_name);
+
+ // required string subscription_name = 3;
+ inline bool has_subscription_name() const;
+ inline void clear_subscription_name();
+ static const int kSubscriptionNameFieldNumber = 3;
+ inline const ::std::string& subscription_name() const;
+ inline void set_subscription_name(const ::std::string& value);
+ inline void set_subscription_name(const char* value);
+ inline void set_subscription_name(const char* value, size_t size);
+ inline ::std::string* mutable_subscription_name();
+ inline ::std::string* release_subscription_name();
+ inline void set_allocated_subscription_name(::std::string* subscription_name);
+
+ // required uint64 consumer_id = 4;
+ inline bool has_consumer_id() const;
+ inline void clear_consumer_id();
+ static const int kConsumerIdFieldNumber = 4;
+ inline ::google::protobuf::uint64 consumer_id() const;
+ inline void set_consumer_id(::google::protobuf::uint64 value);
+
+ // @@protoc_insertion_point(class_scope:pulsar.proto.CommandConsumerStats)
+ private:
+ inline void set_has_request_id();
+ inline void clear_has_request_id();
+ inline void set_has_topic_name();
+ inline void clear_has_topic_name();
+ inline void set_has_subscription_name();
+ inline void clear_has_subscription_name();
+ inline void set_has_consumer_id();
+ inline void clear_has_consumer_id();
+
+ ::std::string _unknown_fields_;
+
+ ::google::protobuf::uint32 _has_bits_[1];
+ mutable int _cached_size_;
+ ::google::protobuf::uint64 request_id_;
+ ::std::string* topic_name_;
+ ::std::string* subscription_name_;
+ ::google::protobuf::uint64 consumer_id_;
+ #ifdef GOOGLE_PROTOBUF_NO_STATIC_INITIALIZER
+ friend void protobuf_AddDesc_PulsarApi_2eproto_impl();
+ #else
+ friend void protobuf_AddDesc_PulsarApi_2eproto();
+ #endif
+ friend void protobuf_AssignDesc_PulsarApi_2eproto();
+ friend void protobuf_ShutdownFile_PulsarApi_2eproto();
+
+ void InitAsDefaultInstance();
+ static CommandConsumerStats* default_instance_;
+};
+// -------------------------------------------------------------------
+
+class CommandConsumerStatsResponse : public ::google::protobuf::MessageLite {
+ public:
+ CommandConsumerStatsResponse();
+ virtual ~CommandConsumerStatsResponse();
+
+ CommandConsumerStatsResponse(const CommandConsumerStatsResponse& from);
+
+ inline CommandConsumerStatsResponse& operator=(const CommandConsumerStatsResponse& from) {
+ CopyFrom(from);
+ return *this;
+ }
+
+ inline const ::std::string& unknown_fields() const {
+ return _unknown_fields_;
+ }
+
+ inline ::std::string* mutable_unknown_fields() {
+ return &_unknown_fields_;
+ }
+
+ static const CommandConsumerStatsResponse& default_instance();
+
+ #ifdef GOOGLE_PROTOBUF_NO_STATIC_INITIALIZER
+ // Returns the internal default instance pointer. This function can
+ // return NULL thus should not be used by the user. This is intended
+ // for Protobuf internal code. Please use default_instance() declared
+ // above instead.
+ static inline const CommandConsumerStatsResponse* internal_default_instance() {
+ return default_instance_;
+ }
+ #endif
+
+ void Swap(CommandConsumerStatsResponse* other);
+
+ // implements Message ----------------------------------------------
+
+ CommandConsumerStatsResponse* New() const;
+ void CheckTypeAndMergeFrom(const ::google::protobuf::MessageLite& from);
+ void CopyFrom(const CommandConsumerStatsResponse& from);
+ void MergeFrom(const CommandConsumerStatsResponse& from);
+ void Clear();
+ bool IsInitialized() const;
+
+ int ByteSize() const;
+ bool MergePartialFromCodedStream(
+ ::google::protobuf::io::CodedInputStream* input);
+ void SerializeWithCachedSizes(
+ ::google::protobuf::io::CodedOutputStream* output) const;
+ void DiscardUnknownFields();
+ int GetCachedSize() const { return _cached_size_; }
+ private:
+ void SharedCtor();
+ void SharedDtor();
+ void SetCachedSize(int size) const;
+ public:
+ ::std::string GetTypeName() const;
+
+ // nested types ----------------------------------------------------
+
+ // accessors -------------------------------------------------------
+
+ // required uint64 request_id = 1;
+ inline bool has_request_id() const;
+ inline void clear_request_id();
+ static const int kRequestIdFieldNumber = 1;
+ inline ::google::protobuf::uint64 request_id() const;
+ inline void set_request_id(::google::protobuf::uint64 value);
+
+ // optional .pulsar.proto.ServerError error_code = 2;
+ inline bool has_error_code() const;
+ inline void clear_error_code();
+ static const int kErrorCodeFieldNumber = 2;
+ inline ::pulsar::proto::ServerError error_code() const;
+ inline void set_error_code(::pulsar::proto::ServerError value);
+
+ // optional string error_message = 3;
+ inline bool has_error_message() const;
+ inline void clear_error_message();
+ static const int kErrorMessageFieldNumber = 3;
+ inline const ::std::string& error_message() const;
+ inline void set_error_message(const ::std::string& value);
+ inline void set_error_message(const char* value);
+ inline void set_error_message(const char* value, size_t size);
+ inline ::std::string* mutable_error_message();
+ inline ::std::string* release_error_message();
+ inline void set_allocated_error_message(::std::string* error_message);
+
+ // optional double msgRateOut = 4;
+ inline bool has_msgrateout() const;
+ inline void clear_msgrateout();
+ static const int kMsgRateOutFieldNumber = 4;
+ inline double msgrateout() const;
+ inline void set_msgrateout(double value);
+
+ // optional double msgThroughputOut = 5;
+ inline bool has_msgthroughputout() const;
+ inline void clear_msgthroughputout();
+ static const int kMsgThroughputOutFieldNumber = 5;
+ inline double msgthroughputout() const;
+ inline void set_msgthroughputout(double value);
+
+ // optional double msgRateRedeliver = 6;
+ inline bool has_msgrateredeliver() const;
+ inline void clear_msgrateredeliver();
+ static const int kMsgRateRedeliverFieldNumber = 6;
+ inline double msgrateredeliver() const;
+ inline void set_msgrateredeliver(double value);
+
+ // optional string consumerName = 7;
+ inline bool has_consumername() const;
+ inline void clear_consumername();
+ static const int kConsumerNameFieldNumber = 7;
+ inline const ::std::string& consumername() const;
+ inline void set_consumername(const ::std::string& value);
+ inline void set_consumername(const char* value);
+ inline void set_consumername(const char* value, size_t size);
+ inline ::std::string* mutable_consumername();
+ inline ::std::string* release_consumername();
+ inline void set_allocated_consumername(::std::string* consumername);
+
+ // optional uint64 availablePermits = 8;
+ inline bool has_availablepermits() const;
+ inline void clear_availablepermits();
+ static const int kAvailablePermitsFieldNumber = 8;
+ inline ::google::protobuf::uint64 availablepermits() const;
+ inline void set_availablepermits(::google::protobuf::uint64 value);
+
+ // optional uint64 unackedMessages = 9;
+ inline bool has_unackedmessages() const;
+ inline void clear_unackedmessages();
+ static const int kUnackedMessagesFieldNumber = 9;
+ inline ::google::protobuf::uint64 unackedmessages() const;
+ inline void set_unackedmessages(::google::protobuf::uint64 value);
+
+ // optional bool blockedConsumerOnUnackedMsgs = 10;
+ inline bool has_blockedconsumeronunackedmsgs() const;
+ inline void clear_blockedconsumeronunackedmsgs();
+ static const int kBlockedConsumerOnUnackedMsgsFieldNumber = 10;
+ inline bool blockedconsumeronunackedmsgs() const;
+ inline void set_blockedconsumeronunackedmsgs(bool value);
+
+ // optional string address = 11;
+ inline bool has_address() const;
+ inline void clear_address();
+ static const int kAddressFieldNumber = 11;
+ inline const ::std::string& address() const;
+ inline void set_address(const ::std::string& value);
+ inline void set_address(const char* value);
+ inline void set_address(const char* value, size_t size);
+ inline ::std::string* mutable_address();
+ inline ::std::string* release_address();
+ inline void set_allocated_address(::std::string* address);
+
+ // optional string connectedSince = 12;
+ inline bool has_connectedsince() const;
+ inline void clear_connectedsince();
+ static const int kConnectedSinceFieldNumber = 12;
+ inline const ::std::string& connectedsince() const;
+ inline void set_connectedsince(const ::std::string& value);
+ inline void set_connectedsince(const char* value);
+ inline void set_connectedsince(const char* value, size_t size);
+ inline ::std::string* mutable_connectedsince();
+ inline ::std::string* release_connectedsince();
+ inline void set_allocated_connectedsince(::std::string* connectedsince);
+
+ // optional string type = 13;
+ inline bool has_type() const;
+ inline void clear_type();
+ static const int kTypeFieldNumber = 13;
+ inline const ::std::string& type() const;
+ inline void set_type(const ::std::string& value);
+ inline void set_type(const char* value);
+ inline void set_type(const char* value, size_t size);
+ inline ::std::string* mutable_type();
+ inline ::std::string* release_type();
+ inline void set_allocated_type(::std::string* type);
+
+ // optional double msgRateExpired = 14;
+ inline bool has_msgrateexpired() const;
+ inline void clear_msgrateexpired();
+ static const int kMsgRateExpiredFieldNumber = 14;
+ inline double msgrateexpired() const;
+ inline void set_msgrateexpired(double value);
+
+ // optional uint64 msgBacklog = 15;
+ inline bool has_msgbacklog() const;
+ inline void clear_msgbacklog();
+ static const int kMsgBacklogFieldNumber = 15;
+ inline ::google::protobuf::uint64 msgbacklog() const;
+ inline void set_msgbacklog(::google::protobuf::uint64 value);
+
+ // @@protoc_insertion_point(class_scope:pulsar.proto.CommandConsumerStatsResponse)
+ private:
+ inline void set_has_request_id();
+ inline void clear_has_request_id();
+ inline void set_has_error_code();
+ inline void clear_has_error_code();
+ inline void set_has_error_message();
+ inline void clear_has_error_message();
+ inline void set_has_msgrateout();
+ inline void clear_has_msgrateout();
+ inline void set_has_msgthroughputout();
+ inline void clear_has_msgthroughputout();
+ inline void set_has_msgrateredeliver();
+ inline void clear_has_msgrateredeliver();
+ inline void set_has_consumername();
+ inline void clear_has_consumername();
+ inline void set_has_availablepermits();
+ inline void clear_has_availablepermits();
+ inline void set_has_unackedmessages();
+ inline void clear_has_unackedmessages();
+ inline void set_has_blockedconsumeronunackedmsgs();
+ inline void clear_has_blockedconsumeronunackedmsgs();
+ inline void set_has_address();
+ inline void clear_has_address();
+ inline void set_has_connectedsince();
+ inline void clear_has_connectedsince();
+ inline void set_has_type();
+ inline void clear_has_type();
+ inline void set_has_msgrateexpired();
+ inline void clear_has_msgrateexpired();
+ inline void set_has_msgbacklog();
+ inline void clear_has_msgbacklog();
+
+ ::std::string _unknown_fields_;
+
+ ::google::protobuf::uint32 _has_bits_[1];
+ mutable int _cached_size_;
+ ::google::protobuf::uint64 request_id_;
+ ::std::string* error_message_;
+ double msgrateout_;
+ double msgthroughputout_;
+ double msgrateredeliver_;
+ int error_code_;
+ bool blockedconsumeronunackedmsgs_;
+ ::std::string* consumername_;
+ ::google::protobuf::uint64 availablepermits_;
+ ::google::protobuf::uint64 unackedmessages_;
+ ::std::string* address_;
+ ::std::string* connectedsince_;
+ ::std::string* type_;
+ double msgrateexpired_;
+ ::google::protobuf::uint64 msgbacklog_;
+ #ifdef GOOGLE_PROTOBUF_NO_STATIC_INITIALIZER
+ friend void protobuf_AddDesc_PulsarApi_2eproto_impl();
+ #else
+ friend void protobuf_AddDesc_PulsarApi_2eproto();
+ #endif
+ friend void protobuf_AssignDesc_PulsarApi_2eproto();
+ friend void protobuf_ShutdownFile_PulsarApi_2eproto();
+
+ void InitAsDefaultInstance();
+ static CommandConsumerStatsResponse* default_instance_;
+};
+// -------------------------------------------------------------------
+
class BaseCommand : public ::google::protobuf::MessageLite {
public:
BaseCommand();
@@ -3525,6 +3921,8 @@ class BaseCommand : public ::google::protobuf::MessageLite {
static const Type PARTITIONED_METADATA_RESPONSE = BaseCommand_Type_PARTITIONED_METADATA_RESPONSE;
static const Type LOOKUP = BaseCommand_Type_LOOKUP;
static const Type LOOKUP_RESPONSE = BaseCommand_Type_LOOKUP_RESPONSE;
+ static const Type CONSUMER_STATS = BaseCommand_Type_CONSUMER_STATS;
+ static const Type CONSUMER_STATS_RESPONSE = BaseCommand_Type_CONSUMER_STATS_RESPONSE;
static inline bool Type_IsValid(int value) {
return BaseCommand_Type_IsValid(value);
}
@@ -3751,6 +4149,24 @@ class BaseCommand : public ::google::protobuf::MessageLite {
inline ::pulsar::proto::CommandLookupTopicResponse* release_lookuptopicresponse();
inline void set_allocated_lookuptopicresponse(::pulsar::proto::CommandLookupTopicResponse* lookuptopicresponse);
+ // optional .pulsar.proto.CommandConsumerStats consumerStats = 25;
+ inline bool has_consumerstats() const;
+ inline void clear_consumerstats();
+ static const int kConsumerStatsFieldNumber = 25;
+ inline const ::pulsar::proto::CommandConsumerStats& consumerstats() const;
+ inline ::pulsar::proto::CommandConsumerStats* mutable_consumerstats();
+ inline ::pulsar::proto::CommandConsumerStats* release_consumerstats();
+ inline void set_allocated_consumerstats(::pulsar::proto::CommandConsumerStats* consumerstats);
+
+ // optional .pulsar.proto.CommandConsumerStatsResponse consumerStatsResponse = 26;
+ inline bool has_consumerstatsresponse() const;
+ inline void clear_consumerstatsresponse();
+ static const int kConsumerStatsResponseFieldNumber = 26;
+ inline const ::pulsar::proto::CommandConsumerStatsResponse& consumerstatsresponse() const;
+ inline ::pulsar::proto::CommandConsumerStatsResponse* mutable_consumerstatsresponse();
+ inline ::pulsar::proto::CommandConsumerStatsResponse* release_consumerstatsresponse();
+ inline void set_allocated_consumerstatsresponse(::pulsar::proto::CommandConsumerStatsResponse* consumerstatsresponse);
+
// @@protoc_insertion_point(class_scope:pulsar.proto.BaseCommand)
private:
inline void set_has_type();
@@ -3801,6 +4217,10 @@ class BaseCommand : public ::google::protobuf::MessageLite {
inline void clear_has_lookuptopic();
inline void set_has_lookuptopicresponse();
inline void clear_has_lookuptopicresponse();
+ inline void set_has_consumerstats();
+ inline void clear_has_consumerstats();
+ inline void set_has_consumerstatsresponse();
+ inline void clear_has_consumerstatsresponse();
::std::string _unknown_fields_;
@@ -3829,6 +4249,8 @@ class BaseCommand : public ::google::protobuf::MessageLite {
::pulsar::proto::CommandPartitionedTopicMetadataResponse* partitionmetadataresponse_;
::pulsar::proto::CommandLookupTopic* lookuptopic_;
::pulsar::proto::CommandLookupTopicResponse* lookuptopicresponse_;
+ ::pulsar::proto::CommandConsumerStats* consumerstats_;
+ ::pulsar::proto::CommandConsumerStatsResponse* consumerstatsresponse_;
int type_;
#ifdef GOOGLE_PROTOBUF_NO_STATIC_INITIALIZER
friend void protobuf_AddDesc_PulsarApi_2eproto_impl();
@@ -7364,103 +7786,932 @@ inline void CommandError::set_allocated_message(::std::string* message) {
// -------------------------------------------------------------------
-// BaseCommand
+// CommandConsumerStats
-// required .pulsar.proto.BaseCommand.Type type = 1;
-inline bool BaseCommand::has_type() const {
+// required uint64 request_id = 1;
+inline bool CommandConsumerStats::has_request_id() const {
return (_has_bits_[0] & 0x00000001u) != 0;
}
-inline void BaseCommand::set_has_type() {
+inline void CommandConsumerStats::set_has_request_id() {
_has_bits_[0] |= 0x00000001u;
}
-inline void BaseCommand::clear_has_type() {
+inline void CommandConsumerStats::clear_has_request_id() {
_has_bits_[0] &= ~0x00000001u;
}
-inline void BaseCommand::clear_type() {
- type_ = 2;
- clear_has_type();
+inline void CommandConsumerStats::clear_request_id() {
+ request_id_ = GOOGLE_ULONGLONG(0);
+ clear_has_request_id();
}
-inline ::pulsar::proto::BaseCommand_Type BaseCommand::type() const {
- // @@protoc_insertion_point(field_get:pulsar.proto.BaseCommand.type)
- return static_cast< ::pulsar::proto::BaseCommand_Type >(type_);
+inline ::google::protobuf::uint64 CommandConsumerStats::request_id() const {
+ // @@protoc_insertion_point(field_get:pulsar.proto.CommandConsumerStats.request_id)
+ return request_id_;
}
-inline void BaseCommand::set_type(::pulsar::proto::BaseCommand_Type value) {
- assert(::pulsar::proto::BaseCommand_Type_IsValid(value));
- set_has_type();
- type_ = value;
- // @@protoc_insertion_point(field_set:pulsar.proto.BaseCommand.type)
+inline void CommandConsumerStats::set_request_id(::google::protobuf::uint64 value) {
+ set_has_request_id();
+ request_id_ = value;
+ // @@protoc_insertion_point(field_set:pulsar.proto.CommandConsumerStats.request_id)
}
-// optional .pulsar.proto.CommandConnect connect = 2;
-inline bool BaseCommand::has_connect() const {
+// required string topic_name = 2;
+inline bool CommandConsumerStats::has_topic_name() const {
return (_has_bits_[0] & 0x00000002u) != 0;
}
-inline void BaseCommand::set_has_connect() {
+inline void CommandConsumerStats::set_has_topic_name() {
_has_bits_[0] |= 0x00000002u;
}
-inline void BaseCommand::clear_has_connect() {
+inline void CommandConsumerStats::clear_has_topic_name() {
_has_bits_[0] &= ~0x00000002u;
}
-inline void BaseCommand::clear_connect() {
- if (connect_ != NULL) connect_->::pulsar::proto::CommandConnect::Clear();
- clear_has_connect();
+inline void CommandConsumerStats::clear_topic_name() {
+ if (topic_name_ != &::google::protobuf::internal::GetEmptyStringAlreadyInited()) {
+ topic_name_->clear();
+ }
+ clear_has_topic_name();
}
-inline const ::pulsar::proto::CommandConnect& BaseCommand::connect() const {
- // @@protoc_insertion_point(field_get:pulsar.proto.BaseCommand.connect)
-#ifdef GOOGLE_PROTOBUF_NO_STATIC_INITIALIZER
- return connect_ != NULL ? *connect_ : *default_instance().connect_;
-#else
- return connect_ != NULL ? *connect_ : *default_instance_->connect_;
-#endif
+inline const ::std::string& CommandConsumerStats::topic_name() const {
+ // @@protoc_insertion_point(field_get:pulsar.proto.CommandConsumerStats.topic_name)
+ return *topic_name_;
}
-inline ::pulsar::proto::CommandConnect* BaseCommand::mutable_connect() {
- set_has_connect();
- if (connect_ == NULL) connect_ = new ::pulsar::proto::CommandConnect;
- // @@protoc_insertion_point(field_mutable:pulsar.proto.BaseCommand.connect)
- return connect_;
+inline void CommandConsumerStats::set_topic_name(const ::std::string& value) {
+ set_has_topic_name();
+ if (topic_name_ == &::google::protobuf::internal::GetEmptyStringAlreadyInited()) {
+ topic_name_ = new ::std::string;
+ }
+ topic_name_->assign(value);
+ // @@protoc_insertion_point(field_set:pulsar.proto.CommandConsumerStats.topic_name)
}
-inline ::pulsar::proto::CommandConnect* BaseCommand::release_connect() {
- clear_has_connect();
- ::pulsar::proto::CommandConnect* temp = connect_;
- connect_ = NULL;
- return temp;
+inline void CommandConsumerStats::set_topic_name(const char* value) {
+ set_has_topic_name();
+ if (topic_name_ == &::google::protobuf::internal::GetEmptyStringAlreadyInited()) {
+ topic_name_ = new ::std::string;
+ }
+ topic_name_->assign(value);
+ // @@protoc_insertion_point(field_set_char:pulsar.proto.CommandConsumerStats.topic_name)
}
-inline void BaseCommand::set_allocated_connect(::pulsar::proto::CommandConnect* connect) {
- delete connect_;
- connect_ = connect;
- if (connect) {
- set_has_connect();
+inline void CommandConsumerStats::set_topic_name(const char* value, size_t size) {
+ set_has_topic_name();
+ if (topic_name_ == &::google::protobuf::internal::GetEmptyStringAlreadyInited()) {
+ topic_name_ = new ::std::string;
+ }
+ topic_name_->assign(reinterpret_cast(value), size);
+ // @@protoc_insertion_point(field_set_pointer:pulsar.proto.CommandConsumerStats.topic_name)
+}
+inline ::std::string* CommandConsumerStats::mutable_topic_name() {
+ set_has_topic_name();
+ if (topic_name_ == &::google::protobuf::internal::GetEmptyStringAlreadyInited()) {
+ topic_name_ = new ::std::string;
+ }
+ // @@protoc_insertion_point(field_mutable:pulsar.proto.CommandConsumerStats.topic_name)
+ return topic_name_;
+}
+inline ::std::string* CommandConsumerStats::release_topic_name() {
+ clear_has_topic_name();
+ if (topic_name_ == &::google::protobuf::internal::GetEmptyStringAlreadyInited()) {
+ return NULL;
} else {
- clear_has_connect();
+ ::std::string* temp = topic_name_;
+ topic_name_ = const_cast< ::std::string*>(&::google::protobuf::internal::GetEmptyStringAlreadyInited());
+ return temp;
}
- // @@protoc_insertion_point(field_set_allocated:pulsar.proto.BaseCommand.connect)
+}
+inline void CommandConsumerStats::set_allocated_topic_name(::std::string* topic_name) {
+ if (topic_name_ != &::google::protobuf::internal::GetEmptyStringAlreadyInited()) {
+ delete topic_name_;
+ }
+ if (topic_name) {
+ set_has_topic_name();
+ topic_name_ = topic_name;
+ } else {
+ clear_has_topic_name();
+ topic_name_ = const_cast< ::std::string*>(&::google::protobuf::internal::GetEmptyStringAlreadyInited());
+ }
+ // @@protoc_insertion_point(field_set_allocated:pulsar.proto.CommandConsumerStats.topic_name)
}
-// optional .pulsar.proto.CommandConnected connected = 3;
-inline bool BaseCommand::has_connected() const {
+// required string subscription_name = 3;
+inline bool CommandConsumerStats::has_subscription_name() const {
return (_has_bits_[0] & 0x00000004u) != 0;
}
-inline void BaseCommand::set_has_connected() {
+inline void CommandConsumerStats::set_has_subscription_name() {
_has_bits_[0] |= 0x00000004u;
}
-inline void BaseCommand::clear_has_connected() {
+inline void CommandConsumerStats::clear_has_subscription_name() {
_has_bits_[0] &= ~0x00000004u;
}
-inline void BaseCommand::clear_connected() {
- if (connected_ != NULL) connected_->::pulsar::proto::CommandConnected::Clear();
- clear_has_connected();
+inline void CommandConsumerStats::clear_subscription_name() {
+ if (subscription_name_ != &::google::protobuf::internal::GetEmptyStringAlreadyInited()) {
+ subscription_name_->clear();
+ }
+ clear_has_subscription_name();
}
-inline const ::pulsar::proto::CommandConnected& BaseCommand::connected() const {
- // @@protoc_insertion_point(field_get:pulsar.proto.BaseCommand.connected)
-#ifdef GOOGLE_PROTOBUF_NO_STATIC_INITIALIZER
- return connected_ != NULL ? *connected_ : *default_instance().connected_;
-#else
- return connected_ != NULL ? *connected_ : *default_instance_->connected_;
-#endif
+inline const ::std::string& CommandConsumerStats::subscription_name() const {
+ // @@protoc_insertion_point(field_get:pulsar.proto.CommandConsumerStats.subscription_name)
+ return *subscription_name_;
}
-inline ::pulsar::proto::CommandConnected* BaseCommand::mutable_connected() {
- set_has_connected();
- if (connected_ == NULL) connected_ = new ::pulsar::proto::CommandConnected;
+inline void CommandConsumerStats::set_subscription_name(const ::std::string& value) {
+ set_has_subscription_name();
+ if (subscription_name_ == &::google::protobuf::internal::GetEmptyStringAlreadyInited()) {
+ subscription_name_ = new ::std::string;
+ }
+ subscription_name_->assign(value);
+ // @@protoc_insertion_point(field_set:pulsar.proto.CommandConsumerStats.subscription_name)
+}
+inline void CommandConsumerStats::set_subscription_name(const char* value) {
+ set_has_subscription_name();
+ if (subscription_name_ == &::google::protobuf::internal::GetEmptyStringAlreadyInited()) {
+ subscription_name_ = new ::std::string;
+ }
+ subscription_name_->assign(value);
+ // @@protoc_insertion_point(field_set_char:pulsar.proto.CommandConsumerStats.subscription_name)
+}
+inline void CommandConsumerStats::set_subscription_name(const char* value, size_t size) {
+ set_has_subscription_name();
+ if (subscription_name_ == &::google::protobuf::internal::GetEmptyStringAlreadyInited()) {
+ subscription_name_ = new ::std::string;
+ }
+ subscription_name_->assign(reinterpret_cast(value), size);
+ // @@protoc_insertion_point(field_set_pointer:pulsar.proto.CommandConsumerStats.subscription_name)
+}
+inline ::std::string* CommandConsumerStats::mutable_subscription_name() {
+ set_has_subscription_name();
+ if (subscription_name_ == &::google::protobuf::internal::GetEmptyStringAlreadyInited()) {
+ subscription_name_ = new ::std::string;
+ }
+ // @@protoc_insertion_point(field_mutable:pulsar.proto.CommandConsumerStats.subscription_name)
+ return subscription_name_;
+}
+inline ::std::string* CommandConsumerStats::release_subscription_name() {
+ clear_has_subscription_name();
+ if (subscription_name_ == &::google::protobuf::internal::GetEmptyStringAlreadyInited()) {
+ return NULL;
+ } else {
+ ::std::string* temp = subscription_name_;
+ subscription_name_ = const_cast< ::std::string*>(&::google::protobuf::internal::GetEmptyStringAlreadyInited());
+ return temp;
+ }
+}
+inline void CommandConsumerStats::set_allocated_subscription_name(::std::string* subscription_name) {
+ if (subscription_name_ != &::google::protobuf::internal::GetEmptyStringAlreadyInited()) {
+ delete subscription_name_;
+ }
+ if (subscription_name) {
+ set_has_subscription_name();
+ subscription_name_ = subscription_name;
+ } else {
+ clear_has_subscription_name();
+ subscription_name_ = const_cast< ::std::string*>(&::google::protobuf::internal::GetEmptyStringAlreadyInited());
+ }
+ // @@protoc_insertion_point(field_set_allocated:pulsar.proto.CommandConsumerStats.subscription_name)
+}
+
+// required uint64 consumer_id = 4;
+inline bool CommandConsumerStats::has_consumer_id() const {
+ return (_has_bits_[0] & 0x00000008u) != 0;
+}
+inline void CommandConsumerStats::set_has_consumer_id() {
+ _has_bits_[0] |= 0x00000008u;
+}
+inline void CommandConsumerStats::clear_has_consumer_id() {
+ _has_bits_[0] &= ~0x00000008u;
+}
+inline void CommandConsumerStats::clear_consumer_id() {
+ consumer_id_ = GOOGLE_ULONGLONG(0);
+ clear_has_consumer_id();
+}
+inline ::google::protobuf::uint64 CommandConsumerStats::consumer_id() const {
+ // @@protoc_insertion_point(field_get:pulsar.proto.CommandConsumerStats.consumer_id)
+ return consumer_id_;
+}
+inline void CommandConsumerStats::set_consumer_id(::google::protobuf::uint64 value) {
+ set_has_consumer_id();
+ consumer_id_ = value;
+ // @@protoc_insertion_point(field_set:pulsar.proto.CommandConsumerStats.consumer_id)
+}
+
+// -------------------------------------------------------------------
+
+// CommandConsumerStatsResponse
+
+// required uint64 request_id = 1;
+inline bool CommandConsumerStatsResponse::has_request_id() const {
+ return (_has_bits_[0] & 0x00000001u) != 0;
+}
+inline void CommandConsumerStatsResponse::set_has_request_id() {
+ _has_bits_[0] |= 0x00000001u;
+}
+inline void CommandConsumerStatsResponse::clear_has_request_id() {
+ _has_bits_[0] &= ~0x00000001u;
+}
+inline void CommandConsumerStatsResponse::clear_request_id() {
+ request_id_ = GOOGLE_ULONGLONG(0);
+ clear_has_request_id();
+}
+inline ::google::protobuf::uint64 CommandConsumerStatsResponse::request_id() const {
+ // @@protoc_insertion_point(field_get:pulsar.proto.CommandConsumerStatsResponse.request_id)
+ return request_id_;
+}
+inline void CommandConsumerStatsResponse::set_request_id(::google::protobuf::uint64 value) {
+ set_has_request_id();
+ request_id_ = value;
+ // @@protoc_insertion_point(field_set:pulsar.proto.CommandConsumerStatsResponse.request_id)
+}
+
+// optional .pulsar.proto.ServerError error_code = 2;
+inline bool CommandConsumerStatsResponse::has_error_code() const {
+ return (_has_bits_[0] & 0x00000002u) != 0;
+}
+inline void CommandConsumerStatsResponse::set_has_error_code() {
+ _has_bits_[0] |= 0x00000002u;
+}
+inline void CommandConsumerStatsResponse::clear_has_error_code() {
+ _has_bits_[0] &= ~0x00000002u;
+}
+inline void CommandConsumerStatsResponse::clear_error_code() {
+ error_code_ = 0;
+ clear_has_error_code();
+}
+inline ::pulsar::proto::ServerError CommandConsumerStatsResponse::error_code() const {
+ // @@protoc_insertion_point(field_get:pulsar.proto.CommandConsumerStatsResponse.error_code)
+ return static_cast< ::pulsar::proto::ServerError >(error_code_);
+}
+inline void CommandConsumerStatsResponse::set_error_code(::pulsar::proto::ServerError value) {
+ assert(::pulsar::proto::ServerError_IsValid(value));
+ set_has_error_code();
+ error_code_ = value;
+ // @@protoc_insertion_point(field_set:pulsar.proto.CommandConsumerStatsResponse.error_code)
+}
+
+// optional string error_message = 3;
+inline bool CommandConsumerStatsResponse::has_error_message() const {
+ return (_has_bits_[0] & 0x00000004u) != 0;
+}
+inline void CommandConsumerStatsResponse::set_has_error_message() {
+ _has_bits_[0] |= 0x00000004u;
+}
+inline void CommandConsumerStatsResponse::clear_has_error_message() {
+ _has_bits_[0] &= ~0x00000004u;
+}
+inline void CommandConsumerStatsResponse::clear_error_message() {
+ if (error_message_ != &::google::protobuf::internal::GetEmptyStringAlreadyInited()) {
+ error_message_->clear();
+ }
+ clear_has_error_message();
+}
+inline const ::std::string& CommandConsumerStatsResponse::error_message() const {
+ // @@protoc_insertion_point(field_get:pulsar.proto.CommandConsumerStatsResponse.error_message)
+ return *error_message_;
+}
+inline void CommandConsumerStatsResponse::set_error_message(const ::std::string& value) {
+ set_has_error_message();
+ if (error_message_ == &::google::protobuf::internal::GetEmptyStringAlreadyInited()) {
+ error_message_ = new ::std::string;
+ }
+ error_message_->assign(value);
+ // @@protoc_insertion_point(field_set:pulsar.proto.CommandConsumerStatsResponse.error_message)
+}
+inline void CommandConsumerStatsResponse::set_error_message(const char* value) {
+ set_has_error_message();
+ if (error_message_ == &::google::protobuf::internal::GetEmptyStringAlreadyInited()) {
+ error_message_ = new ::std::string;
+ }
+ error_message_->assign(value);
+ // @@protoc_insertion_point(field_set_char:pulsar.proto.CommandConsumerStatsResponse.error_message)
+}
+inline void CommandConsumerStatsResponse::set_error_message(const char* value, size_t size) {
+ set_has_error_message();
+ if (error_message_ == &::google::protobuf::internal::GetEmptyStringAlreadyInited()) {
+ error_message_ = new ::std::string;
+ }
+ error_message_->assign(reinterpret_cast(value), size);
+ // @@protoc_insertion_point(field_set_pointer:pulsar.proto.CommandConsumerStatsResponse.error_message)
+}
+inline ::std::string* CommandConsumerStatsResponse::mutable_error_message() {
+ set_has_error_message();
+ if (error_message_ == &::google::protobuf::internal::GetEmptyStringAlreadyInited()) {
+ error_message_ = new ::std::string;
+ }
+ // @@protoc_insertion_point(field_mutable:pulsar.proto.CommandConsumerStatsResponse.error_message)
+ return error_message_;
+}
+inline ::std::string* CommandConsumerStatsResponse::release_error_message() {
+ clear_has_error_message();
+ if (error_message_ == &::google::protobuf::internal::GetEmptyStringAlreadyInited()) {
+ return NULL;
+ } else {
+ ::std::string* temp = error_message_;
+ error_message_ = const_cast< ::std::string*>(&::google::protobuf::internal::GetEmptyStringAlreadyInited());
+ return temp;
+ }
+}
+inline void CommandConsumerStatsResponse::set_allocated_error_message(::std::string* error_message) {
+ if (error_message_ != &::google::protobuf::internal::GetEmptyStringAlreadyInited()) {
+ delete error_message_;
+ }
+ if (error_message) {
+ set_has_error_message();
+ error_message_ = error_message;
+ } else {
+ clear_has_error_message();
+ error_message_ = const_cast< ::std::string*>(&::google::protobuf::internal::GetEmptyStringAlreadyInited());
+ }
+ // @@protoc_insertion_point(field_set_allocated:pulsar.proto.CommandConsumerStatsResponse.error_message)
+}
+
+// optional double msgRateOut = 4;
+inline bool CommandConsumerStatsResponse::has_msgrateout() const {
+ return (_has_bits_[0] & 0x00000008u) != 0;
+}
+inline void CommandConsumerStatsResponse::set_has_msgrateout() {
+ _has_bits_[0] |= 0x00000008u;
+}
+inline void CommandConsumerStatsResponse::clear_has_msgrateout() {
+ _has_bits_[0] &= ~0x00000008u;
+}
+inline void CommandConsumerStatsResponse::clear_msgrateout() {
+ msgrateout_ = 0;
+ clear_has_msgrateout();
+}
+inline double CommandConsumerStatsResponse::msgrateout() const {
+ // @@protoc_insertion_point(field_get:pulsar.proto.CommandConsumerStatsResponse.msgRateOut)
+ return msgrateout_;
+}
+inline void CommandConsumerStatsResponse::set_msgrateout(double value) {
+ set_has_msgrateout();
+ msgrateout_ = value;
+ // @@protoc_insertion_point(field_set:pulsar.proto.CommandConsumerStatsResponse.msgRateOut)
+}
+
+// optional double msgThroughputOut = 5;
+inline bool CommandConsumerStatsResponse::has_msgthroughputout() const {
+ return (_has_bits_[0] & 0x00000010u) != 0;
+}
+inline void CommandConsumerStatsResponse::set_has_msgthroughputout() {
+ _has_bits_[0] |= 0x00000010u;
+}
+inline void CommandConsumerStatsResponse::clear_has_msgthroughputout() {
+ _has_bits_[0] &= ~0x00000010u;
+}
+inline void CommandConsumerStatsResponse::clear_msgthroughputout() {
+ msgthroughputout_ = 0;
+ clear_has_msgthroughputout();
+}
+inline double CommandConsumerStatsResponse::msgthroughputout() const {
+ // @@protoc_insertion_point(field_get:pulsar.proto.CommandConsumerStatsResponse.msgThroughputOut)
+ return msgthroughputout_;
+}
+inline void CommandConsumerStatsResponse::set_msgthroughputout(double value) {
+ set_has_msgthroughputout();
+ msgthroughputout_ = value;
+ // @@protoc_insertion_point(field_set:pulsar.proto.CommandConsumerStatsResponse.msgThroughputOut)
+}
+
+// optional double msgRateRedeliver = 6;
+inline bool CommandConsumerStatsResponse::has_msgrateredeliver() const {
+ return (_has_bits_[0] & 0x00000020u) != 0;
+}
+inline void CommandConsumerStatsResponse::set_has_msgrateredeliver() {
+ _has_bits_[0] |= 0x00000020u;
+}
+inline void CommandConsumerStatsResponse::clear_has_msgrateredeliver() {
+ _has_bits_[0] &= ~0x00000020u;
+}
+inline void CommandConsumerStatsResponse::clear_msgrateredeliver() {
+ msgrateredeliver_ = 0;
+ clear_has_msgrateredeliver();
+}
+inline double CommandConsumerStatsResponse::msgrateredeliver() const {
+ // @@protoc_insertion_point(field_get:pulsar.proto.CommandConsumerStatsResponse.msgRateRedeliver)
+ return msgrateredeliver_;
+}
+inline void CommandConsumerStatsResponse::set_msgrateredeliver(double value) {
+ set_has_msgrateredeliver();
+ msgrateredeliver_ = value;
+ // @@protoc_insertion_point(field_set:pulsar.proto.CommandConsumerStatsResponse.msgRateRedeliver)
+}
+
+// optional string consumerName = 7;
+inline bool CommandConsumerStatsResponse::has_consumername() const {
+ return (_has_bits_[0] & 0x00000040u) != 0;
+}
+inline void CommandConsumerStatsResponse::set_has_consumername() {
+ _has_bits_[0] |= 0x00000040u;
+}
+inline void CommandConsumerStatsResponse::clear_has_consumername() {
+ _has_bits_[0] &= ~0x00000040u;
+}
+inline void CommandConsumerStatsResponse::clear_consumername() {
+ if (consumername_ != &::google::protobuf::internal::GetEmptyStringAlreadyInited()) {
+ consumername_->clear();
+ }
+ clear_has_consumername();
+}
+inline const ::std::string& CommandConsumerStatsResponse::consumername() const {
+ // @@protoc_insertion_point(field_get:pulsar.proto.CommandConsumerStatsResponse.consumerName)
+ return *consumername_;
+}
+inline void CommandConsumerStatsResponse::set_consumername(const ::std::string& value) {
+ set_has_consumername();
+ if (consumername_ == &::google::protobuf::internal::GetEmptyStringAlreadyInited()) {
+ consumername_ = new ::std::string;
+ }
+ consumername_->assign(value);
+ // @@protoc_insertion_point(field_set:pulsar.proto.CommandConsumerStatsResponse.consumerName)
+}
+inline void CommandConsumerStatsResponse::set_consumername(const char* value) {
+ set_has_consumername();
+ if (consumername_ == &::google::protobuf::internal::GetEmptyStringAlreadyInited()) {
+ consumername_ = new ::std::string;
+ }
+ consumername_->assign(value);
+ // @@protoc_insertion_point(field_set_char:pulsar.proto.CommandConsumerStatsResponse.consumerName)
+}
+inline void CommandConsumerStatsResponse::set_consumername(const char* value, size_t size) {
+ set_has_consumername();
+ if (consumername_ == &::google::protobuf::internal::GetEmptyStringAlreadyInited()) {
+ consumername_ = new ::std::string;
+ }
+ consumername_->assign(reinterpret_cast(value), size);
+ // @@protoc_insertion_point(field_set_pointer:pulsar.proto.CommandConsumerStatsResponse.consumerName)
+}
+inline ::std::string* CommandConsumerStatsResponse::mutable_consumername() {
+ set_has_consumername();
+ if (consumername_ == &::google::protobuf::internal::GetEmptyStringAlreadyInited()) {
+ consumername_ = new ::std::string;
+ }
+ // @@protoc_insertion_point(field_mutable:pulsar.proto.CommandConsumerStatsResponse.consumerName)
+ return consumername_;
+}
+inline ::std::string* CommandConsumerStatsResponse::release_consumername() {
+ clear_has_consumername();
+ if (consumername_ == &::google::protobuf::internal::GetEmptyStringAlreadyInited()) {
+ return NULL;
+ } else {
+ ::std::string* temp = consumername_;
+ consumername_ = const_cast< ::std::string*>(&::google::protobuf::internal::GetEmptyStringAlreadyInited());
+ return temp;
+ }
+}
+inline void CommandConsumerStatsResponse::set_allocated_consumername(::std::string* consumername) {
+ if (consumername_ != &::google::protobuf::internal::GetEmptyStringAlreadyInited()) {
+ delete consumername_;
+ }
+ if (consumername) {
+ set_has_consumername();
+ consumername_ = consumername;
+ } else {
+ clear_has_consumername();
+ consumername_ = const_cast< ::std::string*>(&::google::protobuf::internal::GetEmptyStringAlreadyInited());
+ }
+ // @@protoc_insertion_point(field_set_allocated:pulsar.proto.CommandConsumerStatsResponse.consumerName)
+}
+
+// optional uint64 availablePermits = 8;
+inline bool CommandConsumerStatsResponse::has_availablepermits() const {
+ return (_has_bits_[0] & 0x00000080u) != 0;
+}
+inline void CommandConsumerStatsResponse::set_has_availablepermits() {
+ _has_bits_[0] |= 0x00000080u;
+}
+inline void CommandConsumerStatsResponse::clear_has_availablepermits() {
+ _has_bits_[0] &= ~0x00000080u;
+}
+inline void CommandConsumerStatsResponse::clear_availablepermits() {
+ availablepermits_ = GOOGLE_ULONGLONG(0);
+ clear_has_availablepermits();
+}
+inline ::google::protobuf::uint64 CommandConsumerStatsResponse::availablepermits() const {
+ // @@protoc_insertion_point(field_get:pulsar.proto.CommandConsumerStatsResponse.availablePermits)
+ return availablepermits_;
+}
+inline void CommandConsumerStatsResponse::set_availablepermits(::google::protobuf::uint64 value) {
+ set_has_availablepermits();
+ availablepermits_ = value;
+ // @@protoc_insertion_point(field_set:pulsar.proto.CommandConsumerStatsResponse.availablePermits)
+}
+
+// optional uint64 unackedMessages = 9;
+inline bool CommandConsumerStatsResponse::has_unackedmessages() const {
+ return (_has_bits_[0] & 0x00000100u) != 0;
+}
+inline void CommandConsumerStatsResponse::set_has_unackedmessages() {
+ _has_bits_[0] |= 0x00000100u;
+}
+inline void CommandConsumerStatsResponse::clear_has_unackedmessages() {
+ _has_bits_[0] &= ~0x00000100u;
+}
+inline void CommandConsumerStatsResponse::clear_unackedmessages() {
+ unackedmessages_ = GOOGLE_ULONGLONG(0);
+ clear_has_unackedmessages();
+}
+inline ::google::protobuf::uint64 CommandConsumerStatsResponse::unackedmessages() const {
+ // @@protoc_insertion_point(field_get:pulsar.proto.CommandConsumerStatsResponse.unackedMessages)
+ return unackedmessages_;
+}
+inline void CommandConsumerStatsResponse::set_unackedmessages(::google::protobuf::uint64 value) {
+ set_has_unackedmessages();
+ unackedmessages_ = value;
+ // @@protoc_insertion_point(field_set:pulsar.proto.CommandConsumerStatsResponse.unackedMessages)
+}
+
+// optional bool blockedConsumerOnUnackedMsgs = 10;
+inline bool CommandConsumerStatsResponse::has_blockedconsumeronunackedmsgs() const {
+ return (_has_bits_[0] & 0x00000200u) != 0;
+}
+inline void CommandConsumerStatsResponse::set_has_blockedconsumeronunackedmsgs() {
+ _has_bits_[0] |= 0x00000200u;
+}
+inline void CommandConsumerStatsResponse::clear_has_blockedconsumeronunackedmsgs() {
+ _has_bits_[0] &= ~0x00000200u;
+}
+inline void CommandConsumerStatsResponse::clear_blockedconsumeronunackedmsgs() {
+ blockedconsumeronunackedmsgs_ = false;
+ clear_has_blockedconsumeronunackedmsgs();
+}
+inline bool CommandConsumerStatsResponse::blockedconsumeronunackedmsgs() const {
+ // @@protoc_insertion_point(field_get:pulsar.proto.CommandConsumerStatsResponse.blockedConsumerOnUnackedMsgs)
+ return blockedconsumeronunackedmsgs_;
+}
+inline void CommandConsumerStatsResponse::set_blockedconsumeronunackedmsgs(bool value) {
+ set_has_blockedconsumeronunackedmsgs();
+ blockedconsumeronunackedmsgs_ = value;
+ // @@protoc_insertion_point(field_set:pulsar.proto.CommandConsumerStatsResponse.blockedConsumerOnUnackedMsgs)
+}
+
+// optional string address = 11;
+inline bool CommandConsumerStatsResponse::has_address() const {
+ return (_has_bits_[0] & 0x00000400u) != 0;
+}
+inline void CommandConsumerStatsResponse::set_has_address() {
+ _has_bits_[0] |= 0x00000400u;
+}
+inline void CommandConsumerStatsResponse::clear_has_address() {
+ _has_bits_[0] &= ~0x00000400u;
+}
+inline void CommandConsumerStatsResponse::clear_address() {
+ if (address_ != &::google::protobuf::internal::GetEmptyStringAlreadyInited()) {
+ address_->clear();
+ }
+ clear_has_address();
+}
+inline const ::std::string& CommandConsumerStatsResponse::address() const {
+ // @@protoc_insertion_point(field_get:pulsar.proto.CommandConsumerStatsResponse.address)
+ return *address_;
+}
+inline void CommandConsumerStatsResponse::set_address(const ::std::string& value) {
+ set_has_address();
+ if (address_ == &::google::protobuf::internal::GetEmptyStringAlreadyInited()) {
+ address_ = new ::std::string;
+ }
+ address_->assign(value);
+ // @@protoc_insertion_point(field_set:pulsar.proto.CommandConsumerStatsResponse.address)
+}
+inline void CommandConsumerStatsResponse::set_address(const char* value) {
+ set_has_address();
+ if (address_ == &::google::protobuf::internal::GetEmptyStringAlreadyInited()) {
+ address_ = new ::std::string;
+ }
+ address_->assign(value);
+ // @@protoc_insertion_point(field_set_char:pulsar.proto.CommandConsumerStatsResponse.address)
+}
+inline void CommandConsumerStatsResponse::set_address(const char* value, size_t size) {
+ set_has_address();
+ if (address_ == &::google::protobuf::internal::GetEmptyStringAlreadyInited()) {
+ address_ = new ::std::string;
+ }
+ address_->assign(reinterpret_cast(value), size);
+ // @@protoc_insertion_point(field_set_pointer:pulsar.proto.CommandConsumerStatsResponse.address)
+}
+inline ::std::string* CommandConsumerStatsResponse::mutable_address() {
+ set_has_address();
+ if (address_ == &::google::protobuf::internal::GetEmptyStringAlreadyInited()) {
+ address_ = new ::std::string;
+ }
+ // @@protoc_insertion_point(field_mutable:pulsar.proto.CommandConsumerStatsResponse.address)
+ return address_;
+}
+inline ::std::string* CommandConsumerStatsResponse::release_address() {
+ clear_has_address();
+ if (address_ == &::google::protobuf::internal::GetEmptyStringAlreadyInited()) {
+ return NULL;
+ } else {
+ ::std::string* temp = address_;
+ address_ = const_cast< ::std::string*>(&::google::protobuf::internal::GetEmptyStringAlreadyInited());
+ return temp;
+ }
+}
+inline void CommandConsumerStatsResponse::set_allocated_address(::std::string* address) {
+ if (address_ != &::google::protobuf::internal::GetEmptyStringAlreadyInited()) {
+ delete address_;
+ }
+ if (address) {
+ set_has_address();
+ address_ = address;
+ } else {
+ clear_has_address();
+ address_ = const_cast< ::std::string*>(&::google::protobuf::internal::GetEmptyStringAlreadyInited());
+ }
+ // @@protoc_insertion_point(field_set_allocated:pulsar.proto.CommandConsumerStatsResponse.address)
+}
+
+// optional string connectedSince = 12;
+inline bool CommandConsumerStatsResponse::has_connectedsince() const {
+ return (_has_bits_[0] & 0x00000800u) != 0;
+}
+inline void CommandConsumerStatsResponse::set_has_connectedsince() {
+ _has_bits_[0] |= 0x00000800u;
+}
+inline void CommandConsumerStatsResponse::clear_has_connectedsince() {
+ _has_bits_[0] &= ~0x00000800u;
+}
+inline void CommandConsumerStatsResponse::clear_connectedsince() {
+ if (connectedsince_ != &::google::protobuf::internal::GetEmptyStringAlreadyInited()) {
+ connectedsince_->clear();
+ }
+ clear_has_connectedsince();
+}
+inline const ::std::string& CommandConsumerStatsResponse::connectedsince() const {
+ // @@protoc_insertion_point(field_get:pulsar.proto.CommandConsumerStatsResponse.connectedSince)
+ return *connectedsince_;
+}
+inline void CommandConsumerStatsResponse::set_connectedsince(const ::std::string& value) {
+ set_has_connectedsince();
+ if (connectedsince_ == &::google::protobuf::internal::GetEmptyStringAlreadyInited()) {
+ connectedsince_ = new ::std::string;
+ }
+ connectedsince_->assign(value);
+ // @@protoc_insertion_point(field_set:pulsar.proto.CommandConsumerStatsResponse.connectedSince)
+}
+inline void CommandConsumerStatsResponse::set_connectedsince(const char* value) {
+ set_has_connectedsince();
+ if (connectedsince_ == &::google::protobuf::internal::GetEmptyStringAlreadyInited()) {
+ connectedsince_ = new ::std::string;
+ }
+ connectedsince_->assign(value);
+ // @@protoc_insertion_point(field_set_char:pulsar.proto.CommandConsumerStatsResponse.connectedSince)
+}
+inline void CommandConsumerStatsResponse::set_connectedsince(const char* value, size_t size) {
+ set_has_connectedsince();
+ if (connectedsince_ == &::google::protobuf::internal::GetEmptyStringAlreadyInited()) {
+ connectedsince_ = new ::std::string;
+ }
+ connectedsince_->assign(reinterpret_cast(value), size);
+ // @@protoc_insertion_point(field_set_pointer:pulsar.proto.CommandConsumerStatsResponse.connectedSince)
+}
+inline ::std::string* CommandConsumerStatsResponse::mutable_connectedsince() {
+ set_has_connectedsince();
+ if (connectedsince_ == &::google::protobuf::internal::GetEmptyStringAlreadyInited()) {
+ connectedsince_ = new ::std::string;
+ }
+ // @@protoc_insertion_point(field_mutable:pulsar.proto.CommandConsumerStatsResponse.connectedSince)
+ return connectedsince_;
+}
+inline ::std::string* CommandConsumerStatsResponse::release_connectedsince() {
+ clear_has_connectedsince();
+ if (connectedsince_ == &::google::protobuf::internal::GetEmptyStringAlreadyInited()) {
+ return NULL;
+ } else {
+ ::std::string* temp = connectedsince_;
+ connectedsince_ = const_cast< ::std::string*>(&::google::protobuf::internal::GetEmptyStringAlreadyInited());
+ return temp;
+ }
+}
+inline void CommandConsumerStatsResponse::set_allocated_connectedsince(::std::string* connectedsince) {
+ if (connectedsince_ != &::google::protobuf::internal::GetEmptyStringAlreadyInited()) {
+ delete connectedsince_;
+ }
+ if (connectedsince) {
+ set_has_connectedsince();
+ connectedsince_ = connectedsince;
+ } else {
+ clear_has_connectedsince();
+ connectedsince_ = const_cast< ::std::string*>(&::google::protobuf::internal::GetEmptyStringAlreadyInited());
+ }
+ // @@protoc_insertion_point(field_set_allocated:pulsar.proto.CommandConsumerStatsResponse.connectedSince)
+}
+
+// optional string type = 13;
+inline bool CommandConsumerStatsResponse::has_type() const {
+ return (_has_bits_[0] & 0x00001000u) != 0;
+}
+inline void CommandConsumerStatsResponse::set_has_type() {
+ _has_bits_[0] |= 0x00001000u;
+}
+inline void CommandConsumerStatsResponse::clear_has_type() {
+ _has_bits_[0] &= ~0x00001000u;
+}
+inline void CommandConsumerStatsResponse::clear_type() {
+ if (type_ != &::google::protobuf::internal::GetEmptyStringAlreadyInited()) {
+ type_->clear();
+ }
+ clear_has_type();
+}
+inline const ::std::string& CommandConsumerStatsResponse::type() const {
+ // @@protoc_insertion_point(field_get:pulsar.proto.CommandConsumerStatsResponse.type)
+ return *type_;
+}
+inline void CommandConsumerStatsResponse::set_type(const ::std::string& value) {
+ set_has_type();
+ if (type_ == &::google::protobuf::internal::GetEmptyStringAlreadyInited()) {
+ type_ = new ::std::string;
+ }
+ type_->assign(value);
+ // @@protoc_insertion_point(field_set:pulsar.proto.CommandConsumerStatsResponse.type)
+}
+inline void CommandConsumerStatsResponse::set_type(const char* value) {
+ set_has_type();
+ if (type_ == &::google::protobuf::internal::GetEmptyStringAlreadyInited()) {
+ type_ = new ::std::string;
+ }
+ type_->assign(value);
+ // @@protoc_insertion_point(field_set_char:pulsar.proto.CommandConsumerStatsResponse.type)
+}
+inline void CommandConsumerStatsResponse::set_type(const char* value, size_t size) {
+ set_has_type();
+ if (type_ == &::google::protobuf::internal::GetEmptyStringAlreadyInited()) {
+ type_ = new ::std::string;
+ }
+ type_->assign(reinterpret_cast(value), size);
+ // @@protoc_insertion_point(field_set_pointer:pulsar.proto.CommandConsumerStatsResponse.type)
+}
+inline ::std::string* CommandConsumerStatsResponse::mutable_type() {
+ set_has_type();
+ if (type_ == &::google::protobuf::internal::GetEmptyStringAlreadyInited()) {
+ type_ = new ::std::string;
+ }
+ // @@protoc_insertion_point(field_mutable:pulsar.proto.CommandConsumerStatsResponse.type)
+ return type_;
+}
+inline ::std::string* CommandConsumerStatsResponse::release_type() {
+ clear_has_type();
+ if (type_ == &::google::protobuf::internal::GetEmptyStringAlreadyInited()) {
+ return NULL;
+ } else {
+ ::std::string* temp = type_;
+ type_ = const_cast< ::std::string*>(&::google::protobuf::internal::GetEmptyStringAlreadyInited());
+ return temp;
+ }
+}
+inline void CommandConsumerStatsResponse::set_allocated_type(::std::string* type) {
+ if (type_ != &::google::protobuf::internal::GetEmptyStringAlreadyInited()) {
+ delete type_;
+ }
+ if (type) {
+ set_has_type();
+ type_ = type;
+ } else {
+ clear_has_type();
+ type_ = const_cast< ::std::string*>(&::google::protobuf::internal::GetEmptyStringAlreadyInited());
+ }
+ // @@protoc_insertion_point(field_set_allocated:pulsar.proto.CommandConsumerStatsResponse.type)
+}
+
+// optional double msgRateExpired = 14;
+inline bool CommandConsumerStatsResponse::has_msgrateexpired() const {
+ return (_has_bits_[0] & 0x00002000u) != 0;
+}
+inline void CommandConsumerStatsResponse::set_has_msgrateexpired() {
+ _has_bits_[0] |= 0x00002000u;
+}
+inline void CommandConsumerStatsResponse::clear_has_msgrateexpired() {
+ _has_bits_[0] &= ~0x00002000u;
+}
+inline void CommandConsumerStatsResponse::clear_msgrateexpired() {
+ msgrateexpired_ = 0;
+ clear_has_msgrateexpired();
+}
+inline double CommandConsumerStatsResponse::msgrateexpired() const {
+ // @@protoc_insertion_point(field_get:pulsar.proto.CommandConsumerStatsResponse.msgRateExpired)
+ return msgrateexpired_;
+}
+inline void CommandConsumerStatsResponse::set_msgrateexpired(double value) {
+ set_has_msgrateexpired();
+ msgrateexpired_ = value;
+ // @@protoc_insertion_point(field_set:pulsar.proto.CommandConsumerStatsResponse.msgRateExpired)
+}
+
+// optional uint64 msgBacklog = 15;
+inline bool CommandConsumerStatsResponse::has_msgbacklog() const {
+ return (_has_bits_[0] & 0x00004000u) != 0;
+}
+inline void CommandConsumerStatsResponse::set_has_msgbacklog() {
+ _has_bits_[0] |= 0x00004000u;
+}
+inline void CommandConsumerStatsResponse::clear_has_msgbacklog() {
+ _has_bits_[0] &= ~0x00004000u;
+}
+inline void CommandConsumerStatsResponse::clear_msgbacklog() {
+ msgbacklog_ = GOOGLE_ULONGLONG(0);
+ clear_has_msgbacklog();
+}
+inline ::google::protobuf::uint64 CommandConsumerStatsResponse::msgbacklog() const {
+ // @@protoc_insertion_point(field_get:pulsar.proto.CommandConsumerStatsResponse.msgBacklog)
+ return msgbacklog_;
+}
+inline void CommandConsumerStatsResponse::set_msgbacklog(::google::protobuf::uint64 value) {
+ set_has_msgbacklog();
+ msgbacklog_ = value;
+ // @@protoc_insertion_point(field_set:pulsar.proto.CommandConsumerStatsResponse.msgBacklog)
+}
+
+// -------------------------------------------------------------------
+
+// BaseCommand
+
+// required .pulsar.proto.BaseCommand.Type type = 1;
+inline bool BaseCommand::has_type() const {
+ return (_has_bits_[0] & 0x00000001u) != 0;
+}
+inline void BaseCommand::set_has_type() {
+ _has_bits_[0] |= 0x00000001u;
+}
+inline void BaseCommand::clear_has_type() {
+ _has_bits_[0] &= ~0x00000001u;
+}
+inline void BaseCommand::clear_type() {
+ type_ = 2;
+ clear_has_type();
+}
+inline ::pulsar::proto::BaseCommand_Type BaseCommand::type() const {
+ // @@protoc_insertion_point(field_get:pulsar.proto.BaseCommand.type)
+ return static_cast< ::pulsar::proto::BaseCommand_Type >(type_);
+}
+inline void BaseCommand::set_type(::pulsar::proto::BaseCommand_Type value) {
+ assert(::pulsar::proto::BaseCommand_Type_IsValid(value));
+ set_has_type();
+ type_ = value;
+ // @@protoc_insertion_point(field_set:pulsar.proto.BaseCommand.type)
+}
+
+// optional .pulsar.proto.CommandConnect connect = 2;
+inline bool BaseCommand::has_connect() const {
+ return (_has_bits_[0] & 0x00000002u) != 0;
+}
+inline void BaseCommand::set_has_connect() {
+ _has_bits_[0] |= 0x00000002u;
+}
+inline void BaseCommand::clear_has_connect() {
+ _has_bits_[0] &= ~0x00000002u;
+}
+inline void BaseCommand::clear_connect() {
+ if (connect_ != NULL) connect_->::pulsar::proto::CommandConnect::Clear();
+ clear_has_connect();
+}
+inline const ::pulsar::proto::CommandConnect& BaseCommand::connect() const {
+ // @@protoc_insertion_point(field_get:pulsar.proto.BaseCommand.connect)
+#ifdef GOOGLE_PROTOBUF_NO_STATIC_INITIALIZER
+ return connect_ != NULL ? *connect_ : *default_instance().connect_;
+#else
+ return connect_ != NULL ? *connect_ : *default_instance_->connect_;
+#endif
+}
+inline ::pulsar::proto::CommandConnect* BaseCommand::mutable_connect() {
+ set_has_connect();
+ if (connect_ == NULL) connect_ = new ::pulsar::proto::CommandConnect;
+ // @@protoc_insertion_point(field_mutable:pulsar.proto.BaseCommand.connect)
+ return connect_;
+}
+inline ::pulsar::proto::CommandConnect* BaseCommand::release_connect() {
+ clear_has_connect();
+ ::pulsar::proto::CommandConnect* temp = connect_;
+ connect_ = NULL;
+ return temp;
+}
+inline void BaseCommand::set_allocated_connect(::pulsar::proto::CommandConnect* connect) {
+ delete connect_;
+ connect_ = connect;
+ if (connect) {
+ set_has_connect();
+ } else {
+ clear_has_connect();
+ }
+ // @@protoc_insertion_point(field_set_allocated:pulsar.proto.BaseCommand.connect)
+}
+
+// optional .pulsar.proto.CommandConnected connected = 3;
+inline bool BaseCommand::has_connected() const {
+ return (_has_bits_[0] & 0x00000004u) != 0;
+}
+inline void BaseCommand::set_has_connected() {
+ _has_bits_[0] |= 0x00000004u;
+}
+inline void BaseCommand::clear_has_connected() {
+ _has_bits_[0] &= ~0x00000004u;
+}
+inline void BaseCommand::clear_connected() {
+ if (connected_ != NULL) connected_->::pulsar::proto::CommandConnected::Clear();
+ clear_has_connected();
+}
+inline const ::pulsar::proto::CommandConnected& BaseCommand::connected() const {
+ // @@protoc_insertion_point(field_get:pulsar.proto.BaseCommand.connected)
+#ifdef GOOGLE_PROTOBUF_NO_STATIC_INITIALIZER
+ return connected_ != NULL ? *connected_ : *default_instance().connected_;
+#else
+ return connected_ != NULL ? *connected_ : *default_instance_->connected_;
+#endif
+}
+inline ::pulsar::proto::CommandConnected* BaseCommand::mutable_connected() {
+ set_has_connected();
+ if (connected_ == NULL) connected_ = new ::pulsar::proto::CommandConnected;
// @@protoc_insertion_point(field_mutable:pulsar.proto.BaseCommand.connected)
return connected_;
}
@@ -8426,6 +9677,96 @@ inline void BaseCommand::set_allocated_lookuptopicresponse(::pulsar::proto::Comm
// @@protoc_insertion_point(field_set_allocated:pulsar.proto.BaseCommand.lookupTopicResponse)
}
+// optional .pulsar.proto.CommandConsumerStats consumerStats = 25;
+inline bool BaseCommand::has_consumerstats() const {
+ return (_has_bits_[0] & 0x01000000u) != 0;
+}
+inline void BaseCommand::set_has_consumerstats() {
+ _has_bits_[0] |= 0x01000000u;
+}
+inline void BaseCommand::clear_has_consumerstats() {
+ _has_bits_[0] &= ~0x01000000u;
+}
+inline void BaseCommand::clear_consumerstats() {
+ if (consumerstats_ != NULL) consumerstats_->::pulsar::proto::CommandConsumerStats::Clear();
+ clear_has_consumerstats();
+}
+inline const ::pulsar::proto::CommandConsumerStats& BaseCommand::consumerstats() const {
+ // @@protoc_insertion_point(field_get:pulsar.proto.BaseCommand.consumerStats)
+#ifdef GOOGLE_PROTOBUF_NO_STATIC_INITIALIZER
+ return consumerstats_ != NULL ? *consumerstats_ : *default_instance().consumerstats_;
+#else
+ return consumerstats_ != NULL ? *consumerstats_ : *default_instance_->consumerstats_;
+#endif
+}
+inline ::pulsar::proto::CommandConsumerStats* BaseCommand::mutable_consumerstats() {
+ set_has_consumerstats();
+ if (consumerstats_ == NULL) consumerstats_ = new ::pulsar::proto::CommandConsumerStats;
+ // @@protoc_insertion_point(field_mutable:pulsar.proto.BaseCommand.consumerStats)
+ return consumerstats_;
+}
+inline ::pulsar::proto::CommandConsumerStats* BaseCommand::release_consumerstats() {
+ clear_has_consumerstats();
+ ::pulsar::proto::CommandConsumerStats* temp = consumerstats_;
+ consumerstats_ = NULL;
+ return temp;
+}
+inline void BaseCommand::set_allocated_consumerstats(::pulsar::proto::CommandConsumerStats* consumerstats) {
+ delete consumerstats_;
+ consumerstats_ = consumerstats;
+ if (consumerstats) {
+ set_has_consumerstats();
+ } else {
+ clear_has_consumerstats();
+ }
+ // @@protoc_insertion_point(field_set_allocated:pulsar.proto.BaseCommand.consumerStats)
+}
+
+// optional .pulsar.proto.CommandConsumerStatsResponse consumerStatsResponse = 26;
+inline bool BaseCommand::has_consumerstatsresponse() const {
+ return (_has_bits_[0] & 0x02000000u) != 0;
+}
+inline void BaseCommand::set_has_consumerstatsresponse() {
+ _has_bits_[0] |= 0x02000000u;
+}
+inline void BaseCommand::clear_has_consumerstatsresponse() {
+ _has_bits_[0] &= ~0x02000000u;
+}
+inline void BaseCommand::clear_consumerstatsresponse() {
+ if (consumerstatsresponse_ != NULL) consumerstatsresponse_->::pulsar::proto::CommandConsumerStatsResponse::Clear();
+ clear_has_consumerstatsresponse();
+}
+inline const ::pulsar::proto::CommandConsumerStatsResponse& BaseCommand::consumerstatsresponse() const {
+ // @@protoc_insertion_point(field_get:pulsar.proto.BaseCommand.consumerStatsResponse)
+#ifdef GOOGLE_PROTOBUF_NO_STATIC_INITIALIZER
+ return consumerstatsresponse_ != NULL ? *consumerstatsresponse_ : *default_instance().consumerstatsresponse_;
+#else
+ return consumerstatsresponse_ != NULL ? *consumerstatsresponse_ : *default_instance_->consumerstatsresponse_;
+#endif
+}
+inline ::pulsar::proto::CommandConsumerStatsResponse* BaseCommand::mutable_consumerstatsresponse() {
+ set_has_consumerstatsresponse();
+ if (consumerstatsresponse_ == NULL) consumerstatsresponse_ = new ::pulsar::proto::CommandConsumerStatsResponse;
+ // @@protoc_insertion_point(field_mutable:pulsar.proto.BaseCommand.consumerStatsResponse)
+ return consumerstatsresponse_;
+}
+inline ::pulsar::proto::CommandConsumerStatsResponse* BaseCommand::release_consumerstatsresponse() {
+ clear_has_consumerstatsresponse();
+ ::pulsar::proto::CommandConsumerStatsResponse* temp = consumerstatsresponse_;
+ consumerstatsresponse_ = NULL;
+ return temp;
+}
+inline void BaseCommand::set_allocated_consumerstatsresponse(::pulsar::proto::CommandConsumerStatsResponse* consumerstatsresponse) {
+ delete consumerstatsresponse_;
+ consumerstatsresponse_ = consumerstatsresponse;
+ if (consumerstatsresponse) {
+ set_has_consumerstatsresponse();
+ } else {
+ clear_has_consumerstatsresponse();
+ }
+ // @@protoc_insertion_point(field_set_allocated:pulsar.proto.BaseCommand.consumerStatsResponse)
+}
+
// @@protoc_insertion_point(namespace_scope)
diff --git a/pulsar-client-cpp/tests/ConsumerStatsTest.cc b/pulsar-client-cpp/tests/ConsumerStatsTest.cc
new file mode 100644
index 0000000000000..b1bfecba04fd0
--- /dev/null
+++ b/pulsar-client-cpp/tests/ConsumerStatsTest.cc
@@ -0,0 +1,215 @@
+/**
+ * Copyright 2016 Yahoo Inc.
+ *
+ * Licensed 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.
+ */
+
+#include
+#include
+#include
+#include
+#include
+#include "DestinationName.h"
+#include
+#include
+#include "boost/date_time/posix_time/posix_time.hpp"
+#include "CustomRoutingPolicy.h"
+#include
+#include "lib/Future.h"
+#include "lib/Utils.h"
+#include
+#include "LogUtils.h"
+#include "PulsarFriend.h"
+#include
+#include "ConsumerTest.h"
+#include "HttpHelper.h"
+DECLARE_LOG_OBJECT();
+
+using namespace pulsar;
+
+static std::string lookupUrl = "pulsar://localhost:8885";
+static std::string adminUrl = "http://localhost:8765/";
+
+
+TEST(ConsumerStatsTest, testBacklogInfo) {
+ long epochTime=time(NULL);
+ std::string testName="testBacklogInfo-" + boost::lexical_cast(epochTime);
+ Client client(lookupUrl);
+ std::string topicName = "persistent://property/cluster/namespace/" + testName;
+ std::string subName = "subscription-name";
+ Consumer consumer;
+ Promise consumerPromise;
+ client.subscribeAsync(topicName, subName, WaitForCallbackValue(consumerPromise));
+ Future consumerFuture = consumerPromise.getFuture();
+ Result result = consumerFuture.get(consumer);
+ ASSERT_EQ(ResultOk, result);
+
+ // handling dangling subscriptions
+ consumer.unsubscribe();
+ client.subscribe(topicName, subName, consumer);
+
+ // Producing messages
+ Producer producer;
+ int numOfMessages = 10;
+ Promise producerPromise;
+ client.createProducerAsync(topicName, WaitForCallbackValue(producerPromise));
+ Future producerFuture = producerPromise.getFuture();
+ result = producerFuture.get(producer);
+ ASSERT_EQ(ResultOk, result);
+
+ std::string prefix = testName + "-";
+ for (int i = 0; i(i);
+ Message msg = MessageBuilder().build();
+ producer.send(msg);
+ }
+
+ BrokerConsumerStats consumerStats;
+ Result res = consumer.getConsumerStats(consumerStats);
+ ASSERT_EQ(res, ResultOk);
+
+ LOG_DEBUG(consumerStats);
+ ASSERT_EQ(consumerStats.msgBacklog_, numOfMessages);
+
+ for (int i = numOfMessages; i<(numOfMessages*2); i++) {
+ std::string messageContent = prefix + boost::lexical_cast(i);
+ Message msg = MessageBuilder().build();
+ producer.send(msg);
+ }
+
+ usleep(35 * 1000 * 1000);
+ res = consumer.getConsumerStats(consumerStats);
+ ASSERT_EQ(res, ResultOk);
+
+ LOG_DEBUG(consumerStats);
+ ASSERT_EQ(consumerStats.msgBacklog_, 2 * numOfMessages);
+ consumer.unsubscribe();
+}
+
+TEST(ConsumerStatsTest, testFailure) {
+ long epochTime=time(NULL);
+ std::string testName="testFailure-" + boost::lexical_cast(epochTime);
+ Client client(lookupUrl);
+ std::string topicName = "persistent://property/cluster/namespace/" + testName;
+ std::string subName = "subscription-name";
+ Consumer consumer;
+ Promise consumerPromise;
+ BrokerConsumerStats consumerStats;
+ client.subscribeAsync(topicName, subName, WaitForCallbackValue(consumerPromise));
+ ASSERT_NE(ResultOk, consumer.getConsumerStats(consumerStats));
+ Future consumerFuture = consumerPromise.getFuture();
+ Result result = consumerFuture.get(consumer);
+ ASSERT_EQ(ResultOk, result);
+
+ // handling dangling subscriptions
+ consumer.unsubscribe();
+ ASSERT_NE(ResultOk, consumer.getConsumerStats(consumerStats));
+ client.subscribe(topicName, subName, consumer);
+
+ // Producing messages
+ Producer producer;
+ int numOfMessages = 5;
+ Promise producerPromise;
+ client.createProducerAsync(topicName, WaitForCallbackValue(producerPromise));
+ Future producerFuture = producerPromise.getFuture();
+ result = producerFuture.get(producer);
+ ASSERT_EQ(ResultOk, result);
+
+ std::string prefix = testName + "-";
+ for (int i = 0; i(i);
+ Message msg = MessageBuilder().build();
+ producer.send(msg);
+ }
+
+ ASSERT_EQ(ResultOk, consumer.getConsumerStats(consumerStats));
+
+ LOG_DEBUG(consumerStats);
+ ASSERT_EQ(consumerStats.msgBacklog_, numOfMessages);
+
+ consumer.unsubscribe();
+ ASSERT_NE(ResultOk, consumer.getConsumerStats(consumerStats));
+}
+
+TEST(ConsumerStatsTest, testCachingMechanism) {
+ long epochTime=time(NULL);
+ std::string testName="testCachingMechanism-" + boost::lexical_cast(epochTime);
+ Client client(lookupUrl);
+ std::string topicName = "persistent://property/cluster/namespace/" + testName;
+ std::string subName = "subscription-name";
+ Consumer consumer;
+ Promise consumerPromise;
+ BrokerConsumerStats consumerStats;
+ client.subscribeAsync(topicName, subName, WaitForCallbackValue(consumerPromise));
+ ASSERT_NE(ResultOk, consumer.getConsumerStats(consumerStats));
+ Future consumerFuture = consumerPromise.getFuture();
+ Result result = consumerFuture.get(consumer);
+ ASSERT_EQ(ResultOk, result);
+
+ // handling dangling subscriptions
+ consumer.unsubscribe();
+ ASSERT_NE(ResultOk, consumer.getConsumerStats(consumerStats));
+ client.subscribe(topicName, subName, consumer);
+
+ // Producing messages
+ Producer producer;
+ int numOfMessages = 5;
+ Promise producerPromise;
+ client.createProducerAsync(topicName, WaitForCallbackValue(producerPromise));
+ Future producerFuture = producerPromise.getFuture();
+ result = producerFuture.get(producer);
+ ASSERT_EQ(ResultOk, result);
+
+ std::string prefix = testName + "-";
+ for (int i = 0; i(i);
+ Message msg = MessageBuilder().build();
+ producer.send(msg);
+ }
+
+ ASSERT_EQ(ResultOk, consumer.getConsumerStats(consumerStats));
+
+ LOG_DEBUG(consumerStats);
+ ASSERT_EQ(consumerStats.msgBacklog_, numOfMessages);
+
+ for (int i = numOfMessages; i<(numOfMessages*2); i++) {
+ std::string messageContent = prefix + boost::lexical_cast(i);
+ Message msg = MessageBuilder().build();
+ producer.send(msg);
+ }
+
+ LOG_DEBUG("Expecting cached results");
+ ASSERT_TRUE(consumerStats.isValid());
+ ASSERT_EQ(ResultOk, consumer.getConsumerStats(consumerStats));
+ LOG_DEBUG(consumerStats);
+ ASSERT_EQ(consumerStats.msgBacklog_, numOfMessages);
+
+ LOG_DEBUG("Still Expecting cached results");
+ usleep(10 * 1000 * 1000);
+ ASSERT_TRUE(consumerStats.isValid());
+ ASSERT_EQ(ResultOk, consumer.getConsumerStats(consumerStats));
+
+ LOG_DEBUG(consumerStats);
+ ASSERT_EQ(consumerStats.msgBacklog_, numOfMessages);
+
+ LOG_DEBUG("Now expecting new results");
+ usleep(25 * 1000 * 1000);
+ ASSERT_FALSE(consumerStats.isValid());
+ ASSERT_EQ(ResultOk, consumer.getConsumerStats(consumerStats));
+
+ LOG_DEBUG(consumerStats);
+ ASSERT_EQ(consumerStats.msgBacklog_, numOfMessages * 2);
+
+ consumer.unsubscribe();
+ ASSERT_NE(ResultOk, consumer.getConsumerStats(consumerStats));
+}
\ No newline at end of file
diff --git a/pulsar-common/src/main/java/com/yahoo/pulsar/common/api/Commands.java b/pulsar-common/src/main/java/com/yahoo/pulsar/common/api/Commands.java
index 341f956156dcf..ec6e16d093f26 100644
--- a/pulsar-common/src/main/java/com/yahoo/pulsar/common/api/Commands.java
+++ b/pulsar-common/src/main/java/com/yahoo/pulsar/common/api/Commands.java
@@ -21,6 +21,7 @@
import com.google.protobuf.ByteString;
import static com.yahoo.pulsar.checksum.utils.Crc32cChecksum.computeChecksum;
import static com.yahoo.pulsar.checksum.utils.Crc32cChecksum.resumeChecksum;
+
import com.yahoo.pulsar.common.api.proto.PulsarApi;
import com.yahoo.pulsar.common.api.proto.PulsarApi.AuthMethod;
import com.yahoo.pulsar.common.api.proto.PulsarApi.BaseCommand;
@@ -32,6 +33,7 @@
import com.yahoo.pulsar.common.api.proto.PulsarApi.CommandCloseProducer;
import com.yahoo.pulsar.common.api.proto.PulsarApi.CommandConnect;
import com.yahoo.pulsar.common.api.proto.PulsarApi.CommandConnected;
+import com.yahoo.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse;
import com.yahoo.pulsar.common.api.proto.PulsarApi.CommandError;
import com.yahoo.pulsar.common.api.proto.PulsarApi.CommandFlow;
import com.yahoo.pulsar.common.api.proto.PulsarApi.CommandLookupTopic;
@@ -50,8 +52,10 @@
import com.yahoo.pulsar.common.api.proto.PulsarApi.CommandSendReceipt;
import com.yahoo.pulsar.common.api.proto.PulsarApi.CommandSubscribe;
import com.yahoo.pulsar.common.api.proto.PulsarApi.CommandSubscribe.SubType;
+import com.yahoo.pulsar.common.policies.data.ConsumerStats;
import com.yahoo.pulsar.common.api.proto.PulsarApi.CommandSuccess;
import com.yahoo.pulsar.common.api.proto.PulsarApi.CommandUnsubscribe;
+import com.yahoo.pulsar.common.api.proto.PulsarApi.KeyValue;
import com.yahoo.pulsar.common.api.proto.PulsarApi.MessageIdData;
import com.yahoo.pulsar.common.api.proto.PulsarApi.MessageMetadata;
import com.yahoo.pulsar.common.api.proto.PulsarApi.ProtocolVersion;
@@ -503,6 +507,30 @@ public static ByteBuf newRedeliverUnacknowledgedMessages(long consumerId, List RECYCLER = new io.netty.util.Recycler() {
+ protected CommandConsumerStats newObject(Handle handle) {
+ return new CommandConsumerStats(handle);
+ }
+ };
+
+ public void recycle() {
+ this.initFields();
+ this.memoizedIsInitialized = -1;
+ this.bitField0_ = 0;
+ this.memoizedSerializedSize = -1;
+ if (handle != null) { RECYCLER.recycle(this, handle); }
+ }
+
+ private CommandConsumerStats(boolean noInit) {}
+
+ private static final CommandConsumerStats defaultInstance;
+ public static CommandConsumerStats getDefaultInstance() {
+ return defaultInstance;
+ }
+
+ public CommandConsumerStats getDefaultInstanceForType() {
+ return defaultInstance;
+ }
+
+ private int bitField0_;
+ // required uint64 request_id = 1;
+ public static final int REQUEST_ID_FIELD_NUMBER = 1;
+ private long requestId_;
+ public boolean hasRequestId() {
+ return ((bitField0_ & 0x00000001) == 0x00000001);
+ }
+ public long getRequestId() {
+ return requestId_;
+ }
+
+ // required string topic_name = 2;
+ public static final int TOPIC_NAME_FIELD_NUMBER = 2;
+ private java.lang.Object topicName_;
+ public boolean hasTopicName() {
+ return ((bitField0_ & 0x00000002) == 0x00000002);
+ }
+ public String getTopicName() {
+ java.lang.Object ref = topicName_;
+ if (ref instanceof String) {
+ return (String) ref;
+ } else {
+ com.google.protobuf.ByteString bs =
+ (com.google.protobuf.ByteString) ref;
+ String s = bs.toStringUtf8();
+ if (com.google.protobuf.Internal.isValidUtf8(bs)) {
+ topicName_ = s;
+ }
+ return s;
+ }
+ }
+ private com.google.protobuf.ByteString getTopicNameBytes() {
+ java.lang.Object ref = topicName_;
+ if (ref instanceof String) {
+ com.google.protobuf.ByteString b =
+ com.google.protobuf.ByteString.copyFromUtf8((String) ref);
+ topicName_ = b;
+ return b;
+ } else {
+ return (com.google.protobuf.ByteString) ref;
+ }
+ }
+
+ // required string subscription_name = 3;
+ public static final int SUBSCRIPTION_NAME_FIELD_NUMBER = 3;
+ private java.lang.Object subscriptionName_;
+ public boolean hasSubscriptionName() {
+ return ((bitField0_ & 0x00000004) == 0x00000004);
+ }
+ public String getSubscriptionName() {
+ java.lang.Object ref = subscriptionName_;
+ if (ref instanceof String) {
+ return (String) ref;
+ } else {
+ com.google.protobuf.ByteString bs =
+ (com.google.protobuf.ByteString) ref;
+ String s = bs.toStringUtf8();
+ if (com.google.protobuf.Internal.isValidUtf8(bs)) {
+ subscriptionName_ = s;
+ }
+ return s;
+ }
+ }
+ private com.google.protobuf.ByteString getSubscriptionNameBytes() {
+ java.lang.Object ref = subscriptionName_;
+ if (ref instanceof String) {
+ com.google.protobuf.ByteString b =
+ com.google.protobuf.ByteString.copyFromUtf8((String) ref);
+ subscriptionName_ = b;
+ return b;
+ } else {
+ return (com.google.protobuf.ByteString) ref;
+ }
+ }
+
+ // required uint64 consumer_id = 4;
+ public static final int CONSUMER_ID_FIELD_NUMBER = 4;
+ private long consumerId_;
+ public boolean hasConsumerId() {
+ return ((bitField0_ & 0x00000008) == 0x00000008);
+ }
+ public long getConsumerId() {
+ return consumerId_;
+ }
+
+ private void initFields() {
+ requestId_ = 0L;
+ topicName_ = "";
+ subscriptionName_ = "";
+ consumerId_ = 0L;
+ }
+ private byte memoizedIsInitialized = -1;
+ public final boolean isInitialized() {
+ byte isInitialized = memoizedIsInitialized;
+ if (isInitialized != -1) return isInitialized == 1;
+
+ if (!hasRequestId()) {
+ memoizedIsInitialized = 0;
+ return false;
+ }
+ if (!hasTopicName()) {
+ memoizedIsInitialized = 0;
+ return false;
+ }
+ if (!hasSubscriptionName()) {
+ memoizedIsInitialized = 0;
+ return false;
+ }
+ if (!hasConsumerId()) {
+ memoizedIsInitialized = 0;
+ return false;
+ }
+ memoizedIsInitialized = 1;
+ return true;
+ }
+
+ public void writeTo(com.google.protobuf.CodedOutputStream output)
+ throws java.io.IOException {
+ throw new RuntimeException("Cannot use CodedOutputStream");
+ }
+
+ public void writeTo(com.yahoo.pulsar.common.util.protobuf.ByteBufCodedOutputStream output)
+ throws java.io.IOException {
+ getSerializedSize();
+ if (((bitField0_ & 0x00000001) == 0x00000001)) {
+ output.writeUInt64(1, requestId_);
+ }
+ if (((bitField0_ & 0x00000002) == 0x00000002)) {
+ output.writeBytes(2, getTopicNameBytes());
+ }
+ if (((bitField0_ & 0x00000004) == 0x00000004)) {
+ output.writeBytes(3, getSubscriptionNameBytes());
+ }
+ if (((bitField0_ & 0x00000008) == 0x00000008)) {
+ output.writeUInt64(4, consumerId_);
+ }
+ }
+
+ private int memoizedSerializedSize = -1;
+ public int getSerializedSize() {
+ int size = memoizedSerializedSize;
+ if (size != -1) return size;
+
+ size = 0;
+ if (((bitField0_ & 0x00000001) == 0x00000001)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeUInt64Size(1, requestId_);
+ }
+ if (((bitField0_ & 0x00000002) == 0x00000002)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeBytesSize(2, getTopicNameBytes());
+ }
+ if (((bitField0_ & 0x00000004) == 0x00000004)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeBytesSize(3, getSubscriptionNameBytes());
+ }
+ if (((bitField0_ & 0x00000008) == 0x00000008)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeUInt64Size(4, consumerId_);
+ }
+ memoizedSerializedSize = size;
+ return size;
+ }
+
+ private static final long serialVersionUID = 0L;
+ @java.lang.Override
+ protected java.lang.Object writeReplace()
+ throws java.io.ObjectStreamException {
+ return super.writeReplace();
+ }
+
+ public static com.yahoo.pulsar.common.api.proto.PulsarApi.CommandConsumerStats parseFrom(
+ com.google.protobuf.ByteString data)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ throw new RuntimeException("Disabled");
+ }
+ public static com.yahoo.pulsar.common.api.proto.PulsarApi.CommandConsumerStats parseFrom(
+ com.google.protobuf.ByteString data,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ throw new RuntimeException("Disabled");
+ }
+ public static com.yahoo.pulsar.common.api.proto.PulsarApi.CommandConsumerStats parseFrom(byte[] data)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data).buildParsed();
+ }
+ public static com.yahoo.pulsar.common.api.proto.PulsarApi.CommandConsumerStats parseFrom(
+ byte[] data,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data, extensionRegistry)
+ .buildParsed();
+ }
+ public static com.yahoo.pulsar.common.api.proto.PulsarApi.CommandConsumerStats parseFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input).buildParsed();
+ }
+ public static com.yahoo.pulsar.common.api.proto.PulsarApi.CommandConsumerStats parseFrom(
+ java.io.InputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input, extensionRegistry)
+ .buildParsed();
+ }
+ public static com.yahoo.pulsar.common.api.proto.PulsarApi.CommandConsumerStats parseDelimitedFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ Builder builder = newBuilder();
+ if (builder.mergeDelimitedFrom(input)) {
+ return builder.buildParsed();
+ } else {
+ return null;
+ }
+ }
+ public static com.yahoo.pulsar.common.api.proto.PulsarApi.CommandConsumerStats parseDelimitedFrom(
+ java.io.InputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ Builder builder = newBuilder();
+ if (builder.mergeDelimitedFrom(input, extensionRegistry)) {
+ return builder.buildParsed();
+ } else {
+ return null;
+ }
+ }
+ public static com.yahoo.pulsar.common.api.proto.PulsarApi.CommandConsumerStats parseFrom(
+ com.google.protobuf.CodedInputStream input)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input).buildParsed();
+ }
+ public static com.yahoo.pulsar.common.api.proto.PulsarApi.CommandConsumerStats parseFrom(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input, extensionRegistry)
+ .buildParsed();
+ }
+
+ public static Builder newBuilder() { return Builder.create(); }
+ public Builder newBuilderForType() { return newBuilder(); }
+ public static Builder newBuilder(com.yahoo.pulsar.common.api.proto.PulsarApi.CommandConsumerStats prototype) {
+ return newBuilder().mergeFrom(prototype);
+ }
+ public Builder toBuilder() { return newBuilder(this); }
+
+ public static final class Builder extends
+ com.google.protobuf.GeneratedMessageLite.Builder<
+ com.yahoo.pulsar.common.api.proto.PulsarApi.CommandConsumerStats, Builder>
+ implements com.yahoo.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsOrBuilder, com.yahoo.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder {
+ // Construct using com.yahoo.pulsar.common.api.proto.PulsarApi.CommandConsumerStats.newBuilder()
+ private final io.netty.util.Recycler.Handle handle;
+ private Builder(io.netty.util.Recycler.Handle handle) {
+ this.handle = handle;
+ maybeForceBuilderInitialization();
+ }
+ private final static io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() {
+ protected Builder newObject(io.netty.util.Recycler.Handle handle) {
+ return new Builder(handle);
+ }
+ };
+
+ public void recycle() {
+ clear();
+ if (handle != null) {RECYCLER.recycle(this, handle);}
+ }
+
+ private void maybeForceBuilderInitialization() {
+ }
+ private static Builder create() {
+ return RECYCLER.get();
+ }
+
+ public Builder clear() {
+ super.clear();
+ requestId_ = 0L;
+ bitField0_ = (bitField0_ & ~0x00000001);
+ topicName_ = "";
+ bitField0_ = (bitField0_ & ~0x00000002);
+ subscriptionName_ = "";
+ bitField0_ = (bitField0_ & ~0x00000004);
+ consumerId_ = 0L;
+ bitField0_ = (bitField0_ & ~0x00000008);
+ return this;
+ }
+
+ public Builder clone() {
+ return create().mergeFrom(buildPartial());
+ }
+
+ public com.yahoo.pulsar.common.api.proto.PulsarApi.CommandConsumerStats getDefaultInstanceForType() {
+ return com.yahoo.pulsar.common.api.proto.PulsarApi.CommandConsumerStats.getDefaultInstance();
+ }
+
+ public com.yahoo.pulsar.common.api.proto.PulsarApi.CommandConsumerStats build() {
+ com.yahoo.pulsar.common.api.proto.PulsarApi.CommandConsumerStats result = buildPartial();
+ if (!result.isInitialized()) {
+ throw newUninitializedMessageException(result);
+ }
+ return result;
+ }
+
+ private com.yahoo.pulsar.common.api.proto.PulsarApi.CommandConsumerStats buildParsed()
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ com.yahoo.pulsar.common.api.proto.PulsarApi.CommandConsumerStats result = buildPartial();
+ if (!result.isInitialized()) {
+ throw newUninitializedMessageException(
+ result).asInvalidProtocolBufferException();
+ }
+ return result;
+ }
+
+ public com.yahoo.pulsar.common.api.proto.PulsarApi.CommandConsumerStats buildPartial() {
+ com.yahoo.pulsar.common.api.proto.PulsarApi.CommandConsumerStats result = com.yahoo.pulsar.common.api.proto.PulsarApi.CommandConsumerStats.RECYCLER.get();
+ int from_bitField0_ = bitField0_;
+ int to_bitField0_ = 0;
+ if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+ to_bitField0_ |= 0x00000001;
+ }
+ result.requestId_ = requestId_;
+ if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+ to_bitField0_ |= 0x00000002;
+ }
+ result.topicName_ = topicName_;
+ if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
+ to_bitField0_ |= 0x00000004;
+ }
+ result.subscriptionName_ = subscriptionName_;
+ if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
+ to_bitField0_ |= 0x00000008;
+ }
+ result.consumerId_ = consumerId_;
+ result.bitField0_ = to_bitField0_;
+ return result;
+ }
+
+ public Builder mergeFrom(com.yahoo.pulsar.common.api.proto.PulsarApi.CommandConsumerStats other) {
+ if (other == com.yahoo.pulsar.common.api.proto.PulsarApi.CommandConsumerStats.getDefaultInstance()) return this;
+ if (other.hasRequestId()) {
+ setRequestId(other.getRequestId());
+ }
+ if (other.hasTopicName()) {
+ setTopicName(other.getTopicName());
+ }
+ if (other.hasSubscriptionName()) {
+ setSubscriptionName(other.getSubscriptionName());
+ }
+ if (other.hasConsumerId()) {
+ setConsumerId(other.getConsumerId());
+ }
+ return this;
+ }
+
+ public final boolean isInitialized() {
+ if (!hasRequestId()) {
+
+ return false;
+ }
+ if (!hasTopicName()) {
+
+ return false;
+ }
+ if (!hasSubscriptionName()) {
+
+ return false;
+ }
+ if (!hasConsumerId()) {
+
+ return false;
+ }
+ return true;
+ }
+
+ public Builder mergeFrom(com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ throw new java.io.IOException("Merge from CodedInputStream is disabled");
+ }
+ public Builder mergeFrom(
+ com.yahoo.pulsar.common.util.protobuf.ByteBufCodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ while (true) {
+ int tag = input.readTag();
+ switch (tag) {
+ case 0:
+
+ return this;
+ default: {
+ if (!input.skipField(tag)) {
+
+ return this;
+ }
+ break;
+ }
+ case 8: {
+ bitField0_ |= 0x00000001;
+ requestId_ = input.readUInt64();
+ break;
+ }
+ case 18: {
+ bitField0_ |= 0x00000002;
+ topicName_ = input.readBytes();
+ break;
+ }
+ case 26: {
+ bitField0_ |= 0x00000004;
+ subscriptionName_ = input.readBytes();
+ break;
+ }
+ case 32: {
+ bitField0_ |= 0x00000008;
+ consumerId_ = input.readUInt64();
+ break;
+ }
+ }
+ }
+ }
+
+ private int bitField0_;
+
+ // required uint64 request_id = 1;
+ private long requestId_ ;
+ public boolean hasRequestId() {
+ return ((bitField0_ & 0x00000001) == 0x00000001);
+ }
+ public long getRequestId() {
+ return requestId_;
+ }
+ public Builder setRequestId(long value) {
+ bitField0_ |= 0x00000001;
+ requestId_ = value;
+
+ return this;
+ }
+ public Builder clearRequestId() {
+ bitField0_ = (bitField0_ & ~0x00000001);
+ requestId_ = 0L;
+
+ return this;
+ }
+
+ // required string topic_name = 2;
+ private java.lang.Object topicName_ = "";
+ public boolean hasTopicName() {
+ return ((bitField0_ & 0x00000002) == 0x00000002);
+ }
+ public String getTopicName() {
+ java.lang.Object ref = topicName_;
+ if (!(ref instanceof String)) {
+ String s = ((com.google.protobuf.ByteString) ref).toStringUtf8();
+ topicName_ = s;
+ return s;
+ } else {
+ return (String) ref;
+ }
+ }
+ public Builder setTopicName(String value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ bitField0_ |= 0x00000002;
+ topicName_ = value;
+
+ return this;
+ }
+ public Builder clearTopicName() {
+ bitField0_ = (bitField0_ & ~0x00000002);
+ topicName_ = getDefaultInstance().getTopicName();
+
+ return this;
+ }
+ void setTopicName(com.google.protobuf.ByteString value) {
+ bitField0_ |= 0x00000002;
+ topicName_ = value;
+
+ }
+
+ // required string subscription_name = 3;
+ private java.lang.Object subscriptionName_ = "";
+ public boolean hasSubscriptionName() {
+ return ((bitField0_ & 0x00000004) == 0x00000004);
+ }
+ public String getSubscriptionName() {
+ java.lang.Object ref = subscriptionName_;
+ if (!(ref instanceof String)) {
+ String s = ((com.google.protobuf.ByteString) ref).toStringUtf8();
+ subscriptionName_ = s;
+ return s;
+ } else {
+ return (String) ref;
+ }
+ }
+ public Builder setSubscriptionName(String value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ bitField0_ |= 0x00000004;
+ subscriptionName_ = value;
+
+ return this;
+ }
+ public Builder clearSubscriptionName() {
+ bitField0_ = (bitField0_ & ~0x00000004);
+ subscriptionName_ = getDefaultInstance().getSubscriptionName();
+
+ return this;
+ }
+ void setSubscriptionName(com.google.protobuf.ByteString value) {
+ bitField0_ |= 0x00000004;
+ subscriptionName_ = value;
+
+ }
+
+ // required uint64 consumer_id = 4;
+ private long consumerId_ ;
+ public boolean hasConsumerId() {
+ return ((bitField0_ & 0x00000008) == 0x00000008);
+ }
+ public long getConsumerId() {
+ return consumerId_;
+ }
+ public Builder setConsumerId(long value) {
+ bitField0_ |= 0x00000008;
+ consumerId_ = value;
+
+ return this;
+ }
+ public Builder clearConsumerId() {
+ bitField0_ = (bitField0_ & ~0x00000008);
+ consumerId_ = 0L;
+
+ return this;
+ }
+
+ // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandConsumerStats)
+ }
+
+ static {
+ defaultInstance = new CommandConsumerStats(true);
+ defaultInstance.initFields();
+ }
+
+ // @@protoc_insertion_point(class_scope:pulsar.proto.CommandConsumerStats)
+ }
+
+ public interface CommandConsumerStatsResponseOrBuilder
+ extends com.google.protobuf.MessageLiteOrBuilder {
+
+ // required uint64 request_id = 1;
+ boolean hasRequestId();
+ long getRequestId();
+
+ // optional .pulsar.proto.ServerError error_code = 2;
+ boolean hasErrorCode();
+ com.yahoo.pulsar.common.api.proto.PulsarApi.ServerError getErrorCode();
+
+ // optional string error_message = 3;
+ boolean hasErrorMessage();
+ String getErrorMessage();
+
+ // optional double msgRateOut = 4;
+ boolean hasMsgRateOut();
+ double getMsgRateOut();
+
+ // optional double msgThroughputOut = 5;
+ boolean hasMsgThroughputOut();
+ double getMsgThroughputOut();
+
+ // optional double msgRateRedeliver = 6;
+ boolean hasMsgRateRedeliver();
+ double getMsgRateRedeliver();
+
+ // optional string consumerName = 7;
+ boolean hasConsumerName();
+ String getConsumerName();
+
+ // optional uint64 availablePermits = 8;
+ boolean hasAvailablePermits();
+ long getAvailablePermits();
+
+ // optional uint64 unackedMessages = 9;
+ boolean hasUnackedMessages();
+ long getUnackedMessages();
+
+ // optional bool blockedConsumerOnUnackedMsgs = 10;
+ boolean hasBlockedConsumerOnUnackedMsgs();
+ boolean getBlockedConsumerOnUnackedMsgs();
+
+ // optional string address = 11;
+ boolean hasAddress();
+ String getAddress();
+
+ // optional string connectedSince = 12;
+ boolean hasConnectedSince();
+ String getConnectedSince();
+
+ // optional string type = 13;
+ boolean hasType();
+ String getType();
+
+ // optional double msgRateExpired = 14;
+ boolean hasMsgRateExpired();
+ double getMsgRateExpired();
+
+ // optional uint64 msgBacklog = 15;
+ boolean hasMsgBacklog();
+ long getMsgBacklog();
+ }
+ public static final class CommandConsumerStatsResponse extends
+ com.google.protobuf.GeneratedMessageLite
+ implements CommandConsumerStatsResponseOrBuilder, com.yahoo.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage {
+ // Use CommandConsumerStatsResponse.newBuilder() to construct.
+ private io.netty.util.Recycler.Handle handle;
+ private CommandConsumerStatsResponse(io.netty.util.Recycler.Handle handle) {
+ this.handle = handle;
+ }
+
+ private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() {
+ protected CommandConsumerStatsResponse newObject(Handle handle) {
+ return new CommandConsumerStatsResponse(handle);
+ }
+ };
+
+ public void recycle() {
+ this.initFields();
+ this.memoizedIsInitialized = -1;
+ this.bitField0_ = 0;
+ this.memoizedSerializedSize = -1;
+ if (handle != null) { RECYCLER.recycle(this, handle); }
+ }
+
+ private CommandConsumerStatsResponse(boolean noInit) {}
+
+ private static final CommandConsumerStatsResponse defaultInstance;
+ public static CommandConsumerStatsResponse getDefaultInstance() {
+ return defaultInstance;
+ }
+
+ public CommandConsumerStatsResponse getDefaultInstanceForType() {
+ return defaultInstance;
+ }
+
+ private int bitField0_;
+ // required uint64 request_id = 1;
+ public static final int REQUEST_ID_FIELD_NUMBER = 1;
+ private long requestId_;
+ public boolean hasRequestId() {
+ return ((bitField0_ & 0x00000001) == 0x00000001);
+ }
+ public long getRequestId() {
+ return requestId_;
+ }
+
+ // optional .pulsar.proto.ServerError error_code = 2;
+ public static final int ERROR_CODE_FIELD_NUMBER = 2;
+ private com.yahoo.pulsar.common.api.proto.PulsarApi.ServerError errorCode_;
+ public boolean hasErrorCode() {
+ return ((bitField0_ & 0x00000002) == 0x00000002);
+ }
+ public com.yahoo.pulsar.common.api.proto.PulsarApi.ServerError getErrorCode() {
+ return errorCode_;
+ }
+
+ // optional string error_message = 3;
+ public static final int ERROR_MESSAGE_FIELD_NUMBER = 3;
+ private java.lang.Object errorMessage_;
+ public boolean hasErrorMessage() {
+ return ((bitField0_ & 0x00000004) == 0x00000004);
+ }
+ public String getErrorMessage() {
+ java.lang.Object ref = errorMessage_;
+ if (ref instanceof String) {
+ return (String) ref;
+ } else {
+ com.google.protobuf.ByteString bs =
+ (com.google.protobuf.ByteString) ref;
+ String s = bs.toStringUtf8();
+ if (com.google.protobuf.Internal.isValidUtf8(bs)) {
+ errorMessage_ = s;
+ }
+ return s;
+ }
+ }
+ private com.google.protobuf.ByteString getErrorMessageBytes() {
+ java.lang.Object ref = errorMessage_;
+ if (ref instanceof String) {
+ com.google.protobuf.ByteString b =
+ com.google.protobuf.ByteString.copyFromUtf8((String) ref);
+ errorMessage_ = b;
+ return b;
+ } else {
+ return (com.google.protobuf.ByteString) ref;
+ }
+ }
+
+ // optional double msgRateOut = 4;
+ public static final int MSGRATEOUT_FIELD_NUMBER = 4;
+ private double msgRateOut_;
+ public boolean hasMsgRateOut() {
+ return ((bitField0_ & 0x00000008) == 0x00000008);
+ }
+ public double getMsgRateOut() {
+ return msgRateOut_;
+ }
+
+ // optional double msgThroughputOut = 5;
+ public static final int MSGTHROUGHPUTOUT_FIELD_NUMBER = 5;
+ private double msgThroughputOut_;
+ public boolean hasMsgThroughputOut() {
+ return ((bitField0_ & 0x00000010) == 0x00000010);
+ }
+ public double getMsgThroughputOut() {
+ return msgThroughputOut_;
+ }
+
+ // optional double msgRateRedeliver = 6;
+ public static final int MSGRATEREDELIVER_FIELD_NUMBER = 6;
+ private double msgRateRedeliver_;
+ public boolean hasMsgRateRedeliver() {
+ return ((bitField0_ & 0x00000020) == 0x00000020);
+ }
+ public double getMsgRateRedeliver() {
+ return msgRateRedeliver_;
+ }
+
+ // optional string consumerName = 7;
+ public static final int CONSUMERNAME_FIELD_NUMBER = 7;
+ private java.lang.Object consumerName_;
+ public boolean hasConsumerName() {
+ return ((bitField0_ & 0x00000040) == 0x00000040);
+ }
+ public String getConsumerName() {
+ java.lang.Object ref = consumerName_;
+ if (ref instanceof String) {
+ return (String) ref;
+ } else {
+ com.google.protobuf.ByteString bs =
+ (com.google.protobuf.ByteString) ref;
+ String s = bs.toStringUtf8();
+ if (com.google.protobuf.Internal.isValidUtf8(bs)) {
+ consumerName_ = s;
+ }
+ return s;
+ }
+ }
+ private com.google.protobuf.ByteString getConsumerNameBytes() {
+ java.lang.Object ref = consumerName_;
+ if (ref instanceof String) {
+ com.google.protobuf.ByteString b =
+ com.google.protobuf.ByteString.copyFromUtf8((String) ref);
+ consumerName_ = b;
+ return b;
+ } else {
+ return (com.google.protobuf.ByteString) ref;
+ }
+ }
+
+ // optional uint64 availablePermits = 8;
+ public static final int AVAILABLEPERMITS_FIELD_NUMBER = 8;
+ private long availablePermits_;
+ public boolean hasAvailablePermits() {
+ return ((bitField0_ & 0x00000080) == 0x00000080);
+ }
+ public long getAvailablePermits() {
+ return availablePermits_;
+ }
+
+ // optional uint64 unackedMessages = 9;
+ public static final int UNACKEDMESSAGES_FIELD_NUMBER = 9;
+ private long unackedMessages_;
+ public boolean hasUnackedMessages() {
+ return ((bitField0_ & 0x00000100) == 0x00000100);
+ }
+ public long getUnackedMessages() {
+ return unackedMessages_;
+ }
+
+ // optional bool blockedConsumerOnUnackedMsgs = 10;
+ public static final int BLOCKEDCONSUMERONUNACKEDMSGS_FIELD_NUMBER = 10;
+ private boolean blockedConsumerOnUnackedMsgs_;
+ public boolean hasBlockedConsumerOnUnackedMsgs() {
+ return ((bitField0_ & 0x00000200) == 0x00000200);
+ }
+ public boolean getBlockedConsumerOnUnackedMsgs() {
+ return blockedConsumerOnUnackedMsgs_;
+ }
+
+ // optional string address = 11;
+ public static final int ADDRESS_FIELD_NUMBER = 11;
+ private java.lang.Object address_;
+ public boolean hasAddress() {
+ return ((bitField0_ & 0x00000400) == 0x00000400);
+ }
+ public String getAddress() {
+ java.lang.Object ref = address_;
+ if (ref instanceof String) {
+ return (String) ref;
+ } else {
+ com.google.protobuf.ByteString bs =
+ (com.google.protobuf.ByteString) ref;
+ String s = bs.toStringUtf8();
+ if (com.google.protobuf.Internal.isValidUtf8(bs)) {
+ address_ = s;
+ }
+ return s;
+ }
+ }
+ private com.google.protobuf.ByteString getAddressBytes() {
+ java.lang.Object ref = address_;
+ if (ref instanceof String) {
+ com.google.protobuf.ByteString b =
+ com.google.protobuf.ByteString.copyFromUtf8((String) ref);
+ address_ = b;
+ return b;
+ } else {
+ return (com.google.protobuf.ByteString) ref;
+ }
+ }
+
+ // optional string connectedSince = 12;
+ public static final int CONNECTEDSINCE_FIELD_NUMBER = 12;
+ private java.lang.Object connectedSince_;
+ public boolean hasConnectedSince() {
+ return ((bitField0_ & 0x00000800) == 0x00000800);
+ }
+ public String getConnectedSince() {
+ java.lang.Object ref = connectedSince_;
+ if (ref instanceof String) {
+ return (String) ref;
+ } else {
+ com.google.protobuf.ByteString bs =
+ (com.google.protobuf.ByteString) ref;
+ String s = bs.toStringUtf8();
+ if (com.google.protobuf.Internal.isValidUtf8(bs)) {
+ connectedSince_ = s;
+ }
+ return s;
+ }
+ }
+ private com.google.protobuf.ByteString getConnectedSinceBytes() {
+ java.lang.Object ref = connectedSince_;
+ if (ref instanceof String) {
+ com.google.protobuf.ByteString b =
+ com.google.protobuf.ByteString.copyFromUtf8((String) ref);
+ connectedSince_ = b;
+ return b;
+ } else {
+ return (com.google.protobuf.ByteString) ref;
+ }
+ }
+
+ // optional string type = 13;
+ public static final int TYPE_FIELD_NUMBER = 13;
+ private java.lang.Object type_;
+ public boolean hasType() {
+ return ((bitField0_ & 0x00001000) == 0x00001000);
+ }
+ public String getType() {
+ java.lang.Object ref = type_;
+ if (ref instanceof String) {
+ return (String) ref;
+ } else {
+ com.google.protobuf.ByteString bs =
+ (com.google.protobuf.ByteString) ref;
+ String s = bs.toStringUtf8();
+ if (com.google.protobuf.Internal.isValidUtf8(bs)) {
+ type_ = s;
+ }
+ return s;
+ }
+ }
+ private com.google.protobuf.ByteString getTypeBytes() {
+ java.lang.Object ref = type_;
+ if (ref instanceof String) {
+ com.google.protobuf.ByteString b =
+ com.google.protobuf.ByteString.copyFromUtf8((String) ref);
+ type_ = b;
+ return b;
+ } else {
+ return (com.google.protobuf.ByteString) ref;
+ }
+ }
+
+ // optional double msgRateExpired = 14;
+ public static final int MSGRATEEXPIRED_FIELD_NUMBER = 14;
+ private double msgRateExpired_;
+ public boolean hasMsgRateExpired() {
+ return ((bitField0_ & 0x00002000) == 0x00002000);
+ }
+ public double getMsgRateExpired() {
+ return msgRateExpired_;
+ }
+
+ // optional uint64 msgBacklog = 15;
+ public static final int MSGBACKLOG_FIELD_NUMBER = 15;
+ private long msgBacklog_;
+ public boolean hasMsgBacklog() {
+ return ((bitField0_ & 0x00004000) == 0x00004000);
+ }
+ public long getMsgBacklog() {
+ return msgBacklog_;
+ }
+
+ private void initFields() {
+ requestId_ = 0L;
+ errorCode_ = com.yahoo.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError;
+ errorMessage_ = "";
+ msgRateOut_ = 0D;
+ msgThroughputOut_ = 0D;
+ msgRateRedeliver_ = 0D;
+ consumerName_ = "";
+ availablePermits_ = 0L;
+ unackedMessages_ = 0L;
+ blockedConsumerOnUnackedMsgs_ = false;
+ address_ = "";
+ connectedSince_ = "";
+ type_ = "";
+ msgRateExpired_ = 0D;
+ msgBacklog_ = 0L;
+ }
+ private byte memoizedIsInitialized = -1;
+ public final boolean isInitialized() {
+ byte isInitialized = memoizedIsInitialized;
+ if (isInitialized != -1) return isInitialized == 1;
+
+ if (!hasRequestId()) {
+ memoizedIsInitialized = 0;
+ return false;
+ }
+ memoizedIsInitialized = 1;
+ return true;
+ }
+
+ public void writeTo(com.google.protobuf.CodedOutputStream output)
+ throws java.io.IOException {
+ throw new RuntimeException("Cannot use CodedOutputStream");
+ }
+
+ public void writeTo(com.yahoo.pulsar.common.util.protobuf.ByteBufCodedOutputStream output)
+ throws java.io.IOException {
+ getSerializedSize();
+ if (((bitField0_ & 0x00000001) == 0x00000001)) {
+ output.writeUInt64(1, requestId_);
+ }
+ if (((bitField0_ & 0x00000002) == 0x00000002)) {
+ output.writeEnum(2, errorCode_.getNumber());
+ }
+ if (((bitField0_ & 0x00000004) == 0x00000004)) {
+ output.writeBytes(3, getErrorMessageBytes());
+ }
+ if (((bitField0_ & 0x00000008) == 0x00000008)) {
+ output.writeDouble(4, msgRateOut_);
+ }
+ if (((bitField0_ & 0x00000010) == 0x00000010)) {
+ output.writeDouble(5, msgThroughputOut_);
+ }
+ if (((bitField0_ & 0x00000020) == 0x00000020)) {
+ output.writeDouble(6, msgRateRedeliver_);
+ }
+ if (((bitField0_ & 0x00000040) == 0x00000040)) {
+ output.writeBytes(7, getConsumerNameBytes());
+ }
+ if (((bitField0_ & 0x00000080) == 0x00000080)) {
+ output.writeUInt64(8, availablePermits_);
+ }
+ if (((bitField0_ & 0x00000100) == 0x00000100)) {
+ output.writeUInt64(9, unackedMessages_);
+ }
+ if (((bitField0_ & 0x00000200) == 0x00000200)) {
+ output.writeBool(10, blockedConsumerOnUnackedMsgs_);
+ }
+ if (((bitField0_ & 0x00000400) == 0x00000400)) {
+ output.writeBytes(11, getAddressBytes());
+ }
+ if (((bitField0_ & 0x00000800) == 0x00000800)) {
+ output.writeBytes(12, getConnectedSinceBytes());
+ }
+ if (((bitField0_ & 0x00001000) == 0x00001000)) {
+ output.writeBytes(13, getTypeBytes());
+ }
+ if (((bitField0_ & 0x00002000) == 0x00002000)) {
+ output.writeDouble(14, msgRateExpired_);
+ }
+ if (((bitField0_ & 0x00004000) == 0x00004000)) {
+ output.writeUInt64(15, msgBacklog_);
+ }
+ }
+
+ private int memoizedSerializedSize = -1;
+ public int getSerializedSize() {
+ int size = memoizedSerializedSize;
+ if (size != -1) return size;
+
+ size = 0;
+ if (((bitField0_ & 0x00000001) == 0x00000001)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeUInt64Size(1, requestId_);
+ }
+ if (((bitField0_ & 0x00000002) == 0x00000002)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeEnumSize(2, errorCode_.getNumber());
+ }
+ if (((bitField0_ & 0x00000004) == 0x00000004)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeBytesSize(3, getErrorMessageBytes());
+ }
+ if (((bitField0_ & 0x00000008) == 0x00000008)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeDoubleSize(4, msgRateOut_);
+ }
+ if (((bitField0_ & 0x00000010) == 0x00000010)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeDoubleSize(5, msgThroughputOut_);
+ }
+ if (((bitField0_ & 0x00000020) == 0x00000020)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeDoubleSize(6, msgRateRedeliver_);
+ }
+ if (((bitField0_ & 0x00000040) == 0x00000040)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeBytesSize(7, getConsumerNameBytes());
+ }
+ if (((bitField0_ & 0x00000080) == 0x00000080)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeUInt64Size(8, availablePermits_);
+ }
+ if (((bitField0_ & 0x00000100) == 0x00000100)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeUInt64Size(9, unackedMessages_);
+ }
+ if (((bitField0_ & 0x00000200) == 0x00000200)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeBoolSize(10, blockedConsumerOnUnackedMsgs_);
+ }
+ if (((bitField0_ & 0x00000400) == 0x00000400)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeBytesSize(11, getAddressBytes());
+ }
+ if (((bitField0_ & 0x00000800) == 0x00000800)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeBytesSize(12, getConnectedSinceBytes());
+ }
+ if (((bitField0_ & 0x00001000) == 0x00001000)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeBytesSize(13, getTypeBytes());
+ }
+ if (((bitField0_ & 0x00002000) == 0x00002000)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeDoubleSize(14, msgRateExpired_);
+ }
+ if (((bitField0_ & 0x00004000) == 0x00004000)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeUInt64Size(15, msgBacklog_);
+ }
+ memoizedSerializedSize = size;
+ return size;
+ }
+
+ private static final long serialVersionUID = 0L;
+ @java.lang.Override
+ protected java.lang.Object writeReplace()
+ throws java.io.ObjectStreamException {
+ return super.writeReplace();
+ }
+
+ public static com.yahoo.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse parseFrom(
+ com.google.protobuf.ByteString data)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ throw new RuntimeException("Disabled");
+ }
+ public static com.yahoo.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse parseFrom(
+ com.google.protobuf.ByteString data,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ throw new RuntimeException("Disabled");
+ }
+ public static com.yahoo.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse parseFrom(byte[] data)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data).buildParsed();
+ }
+ public static com.yahoo.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse parseFrom(
+ byte[] data,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data, extensionRegistry)
+ .buildParsed();
+ }
+ public static com.yahoo.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse parseFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input).buildParsed();
+ }
+ public static com.yahoo.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse parseFrom(
+ java.io.InputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input, extensionRegistry)
+ .buildParsed();
+ }
+ public static com.yahoo.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse parseDelimitedFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ Builder builder = newBuilder();
+ if (builder.mergeDelimitedFrom(input)) {
+ return builder.buildParsed();
+ } else {
+ return null;
+ }
+ }
+ public static com.yahoo.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse parseDelimitedFrom(
+ java.io.InputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ Builder builder = newBuilder();
+ if (builder.mergeDelimitedFrom(input, extensionRegistry)) {
+ return builder.buildParsed();
+ } else {
+ return null;
+ }
+ }
+ public static com.yahoo.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse parseFrom(
+ com.google.protobuf.CodedInputStream input)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input).buildParsed();
+ }
+ public static com.yahoo.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse parseFrom(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input, extensionRegistry)
+ .buildParsed();
+ }
+
+ public static Builder newBuilder() { return Builder.create(); }
+ public Builder newBuilderForType() { return newBuilder(); }
+ public static Builder newBuilder(com.yahoo.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse prototype) {
+ return newBuilder().mergeFrom(prototype);
+ }
+ public Builder toBuilder() { return newBuilder(this); }
+
+ public static final class Builder extends
+ com.google.protobuf.GeneratedMessageLite.Builder<
+ com.yahoo.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse, Builder>
+ implements com.yahoo.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponseOrBuilder, com.yahoo.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder {
+ // Construct using com.yahoo.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse.newBuilder()
+ private final io.netty.util.Recycler.Handle handle;
+ private Builder(io.netty.util.Recycler.Handle handle) {
+ this.handle = handle;
+ maybeForceBuilderInitialization();
+ }
+ private final static io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() {
+ protected Builder newObject(io.netty.util.Recycler.Handle handle) {
+ return new Builder(handle);
+ }
+ };
+
+ public void recycle() {
+ clear();
+ if (handle != null) {RECYCLER.recycle(this, handle);}
+ }
+
+ private void maybeForceBuilderInitialization() {
+ }
+ private static Builder create() {
+ return RECYCLER.get();
+ }
+
+ public Builder clear() {
+ super.clear();
+ requestId_ = 0L;
+ bitField0_ = (bitField0_ & ~0x00000001);
+ errorCode_ = com.yahoo.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError;
+ bitField0_ = (bitField0_ & ~0x00000002);
+ errorMessage_ = "";
+ bitField0_ = (bitField0_ & ~0x00000004);
+ msgRateOut_ = 0D;
+ bitField0_ = (bitField0_ & ~0x00000008);
+ msgThroughputOut_ = 0D;
+ bitField0_ = (bitField0_ & ~0x00000010);
+ msgRateRedeliver_ = 0D;
+ bitField0_ = (bitField0_ & ~0x00000020);
+ consumerName_ = "";
+ bitField0_ = (bitField0_ & ~0x00000040);
+ availablePermits_ = 0L;
+ bitField0_ = (bitField0_ & ~0x00000080);
+ unackedMessages_ = 0L;
+ bitField0_ = (bitField0_ & ~0x00000100);
+ blockedConsumerOnUnackedMsgs_ = false;
+ bitField0_ = (bitField0_ & ~0x00000200);
+ address_ = "";
+ bitField0_ = (bitField0_ & ~0x00000400);
+ connectedSince_ = "";
+ bitField0_ = (bitField0_ & ~0x00000800);
+ type_ = "";
+ bitField0_ = (bitField0_ & ~0x00001000);
+ msgRateExpired_ = 0D;
+ bitField0_ = (bitField0_ & ~0x00002000);
+ msgBacklog_ = 0L;
+ bitField0_ = (bitField0_ & ~0x00004000);
+ return this;
+ }
+
+ public Builder clone() {
+ return create().mergeFrom(buildPartial());
+ }
+
+ public com.yahoo.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse getDefaultInstanceForType() {
+ return com.yahoo.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse.getDefaultInstance();
+ }
+
+ public com.yahoo.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse build() {
+ com.yahoo.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse result = buildPartial();
+ if (!result.isInitialized()) {
+ throw newUninitializedMessageException(result);
+ }
+ return result;
+ }
+
+ private com.yahoo.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse buildParsed()
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ com.yahoo.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse result = buildPartial();
+ if (!result.isInitialized()) {
+ throw newUninitializedMessageException(
+ result).asInvalidProtocolBufferException();
+ }
+ return result;
+ }
+
+ public com.yahoo.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse buildPartial() {
+ com.yahoo.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse result = com.yahoo.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse.RECYCLER.get();
+ int from_bitField0_ = bitField0_;
+ int to_bitField0_ = 0;
+ if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+ to_bitField0_ |= 0x00000001;
+ }
+ result.requestId_ = requestId_;
+ if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+ to_bitField0_ |= 0x00000002;
+ }
+ result.errorCode_ = errorCode_;
+ if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
+ to_bitField0_ |= 0x00000004;
+ }
+ result.errorMessage_ = errorMessage_;
+ if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
+ to_bitField0_ |= 0x00000008;
+ }
+ result.msgRateOut_ = msgRateOut_;
+ if (((from_bitField0_ & 0x00000010) == 0x00000010)) {
+ to_bitField0_ |= 0x00000010;
+ }
+ result.msgThroughputOut_ = msgThroughputOut_;
+ if (((from_bitField0_ & 0x00000020) == 0x00000020)) {
+ to_bitField0_ |= 0x00000020;
+ }
+ result.msgRateRedeliver_ = msgRateRedeliver_;
+ if (((from_bitField0_ & 0x00000040) == 0x00000040)) {
+ to_bitField0_ |= 0x00000040;
+ }
+ result.consumerName_ = consumerName_;
+ if (((from_bitField0_ & 0x00000080) == 0x00000080)) {
+ to_bitField0_ |= 0x00000080;
+ }
+ result.availablePermits_ = availablePermits_;
+ if (((from_bitField0_ & 0x00000100) == 0x00000100)) {
+ to_bitField0_ |= 0x00000100;
+ }
+ result.unackedMessages_ = unackedMessages_;
+ if (((from_bitField0_ & 0x00000200) == 0x00000200)) {
+ to_bitField0_ |= 0x00000200;
+ }
+ result.blockedConsumerOnUnackedMsgs_ = blockedConsumerOnUnackedMsgs_;
+ if (((from_bitField0_ & 0x00000400) == 0x00000400)) {
+ to_bitField0_ |= 0x00000400;
+ }
+ result.address_ = address_;
+ if (((from_bitField0_ & 0x00000800) == 0x00000800)) {
+ to_bitField0_ |= 0x00000800;
+ }
+ result.connectedSince_ = connectedSince_;
+ if (((from_bitField0_ & 0x00001000) == 0x00001000)) {
+ to_bitField0_ |= 0x00001000;
+ }
+ result.type_ = type_;
+ if (((from_bitField0_ & 0x00002000) == 0x00002000)) {
+ to_bitField0_ |= 0x00002000;
+ }
+ result.msgRateExpired_ = msgRateExpired_;
+ if (((from_bitField0_ & 0x00004000) == 0x00004000)) {
+ to_bitField0_ |= 0x00004000;
+ }
+ result.msgBacklog_ = msgBacklog_;
+ result.bitField0_ = to_bitField0_;
+ return result;
+ }
+
+ public Builder mergeFrom(com.yahoo.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse other) {
+ if (other == com.yahoo.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse.getDefaultInstance()) return this;
+ if (other.hasRequestId()) {
+ setRequestId(other.getRequestId());
+ }
+ if (other.hasErrorCode()) {
+ setErrorCode(other.getErrorCode());
+ }
+ if (other.hasErrorMessage()) {
+ setErrorMessage(other.getErrorMessage());
+ }
+ if (other.hasMsgRateOut()) {
+ setMsgRateOut(other.getMsgRateOut());
+ }
+ if (other.hasMsgThroughputOut()) {
+ setMsgThroughputOut(other.getMsgThroughputOut());
+ }
+ if (other.hasMsgRateRedeliver()) {
+ setMsgRateRedeliver(other.getMsgRateRedeliver());
+ }
+ if (other.hasConsumerName()) {
+ setConsumerName(other.getConsumerName());
+ }
+ if (other.hasAvailablePermits()) {
+ setAvailablePermits(other.getAvailablePermits());
+ }
+ if (other.hasUnackedMessages()) {
+ setUnackedMessages(other.getUnackedMessages());
+ }
+ if (other.hasBlockedConsumerOnUnackedMsgs()) {
+ setBlockedConsumerOnUnackedMsgs(other.getBlockedConsumerOnUnackedMsgs());
+ }
+ if (other.hasAddress()) {
+ setAddress(other.getAddress());
+ }
+ if (other.hasConnectedSince()) {
+ setConnectedSince(other.getConnectedSince());
+ }
+ if (other.hasType()) {
+ setType(other.getType());
+ }
+ if (other.hasMsgRateExpired()) {
+ setMsgRateExpired(other.getMsgRateExpired());
+ }
+ if (other.hasMsgBacklog()) {
+ setMsgBacklog(other.getMsgBacklog());
+ }
+ return this;
+ }
+
+ public final boolean isInitialized() {
+ if (!hasRequestId()) {
+
+ return false;
+ }
+ return true;
+ }
+
+ public Builder mergeFrom(com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ throw new java.io.IOException("Merge from CodedInputStream is disabled");
+ }
+ public Builder mergeFrom(
+ com.yahoo.pulsar.common.util.protobuf.ByteBufCodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ while (true) {
+ int tag = input.readTag();
+ switch (tag) {
+ case 0:
+
+ return this;
+ default: {
+ if (!input.skipField(tag)) {
+
+ return this;
+ }
+ break;
+ }
+ case 8: {
+ bitField0_ |= 0x00000001;
+ requestId_ = input.readUInt64();
+ break;
+ }
+ case 16: {
+ int rawValue = input.readEnum();
+ com.yahoo.pulsar.common.api.proto.PulsarApi.ServerError value = com.yahoo.pulsar.common.api.proto.PulsarApi.ServerError.valueOf(rawValue);
+ if (value != null) {
+ bitField0_ |= 0x00000002;
+ errorCode_ = value;
+ }
+ break;
+ }
+ case 26: {
+ bitField0_ |= 0x00000004;
+ errorMessage_ = input.readBytes();
+ break;
+ }
+ case 33: {
+ bitField0_ |= 0x00000008;
+ msgRateOut_ = input.readDouble();
+ break;
+ }
+ case 41: {
+ bitField0_ |= 0x00000010;
+ msgThroughputOut_ = input.readDouble();
+ break;
+ }
+ case 49: {
+ bitField0_ |= 0x00000020;
+ msgRateRedeliver_ = input.readDouble();
+ break;
+ }
+ case 58: {
+ bitField0_ |= 0x00000040;
+ consumerName_ = input.readBytes();
+ break;
+ }
+ case 64: {
+ bitField0_ |= 0x00000080;
+ availablePermits_ = input.readUInt64();
+ break;
+ }
+ case 72: {
+ bitField0_ |= 0x00000100;
+ unackedMessages_ = input.readUInt64();
+ break;
+ }
+ case 80: {
+ bitField0_ |= 0x00000200;
+ blockedConsumerOnUnackedMsgs_ = input.readBool();
+ break;
+ }
+ case 90: {
+ bitField0_ |= 0x00000400;
+ address_ = input.readBytes();
+ break;
+ }
+ case 98: {
+ bitField0_ |= 0x00000800;
+ connectedSince_ = input.readBytes();
+ break;
+ }
+ case 106: {
+ bitField0_ |= 0x00001000;
+ type_ = input.readBytes();
+ break;
+ }
+ case 113: {
+ bitField0_ |= 0x00002000;
+ msgRateExpired_ = input.readDouble();
+ break;
+ }
+ case 120: {
+ bitField0_ |= 0x00004000;
+ msgBacklog_ = input.readUInt64();
+ break;
+ }
+ }
+ }
+ }
+
+ private int bitField0_;
+
+ // required uint64 request_id = 1;
+ private long requestId_ ;
+ public boolean hasRequestId() {
+ return ((bitField0_ & 0x00000001) == 0x00000001);
+ }
+ public long getRequestId() {
+ return requestId_;
+ }
+ public Builder setRequestId(long value) {
+ bitField0_ |= 0x00000001;
+ requestId_ = value;
+
+ return this;
+ }
+ public Builder clearRequestId() {
+ bitField0_ = (bitField0_ & ~0x00000001);
+ requestId_ = 0L;
+
+ return this;
+ }
+
+ // optional .pulsar.proto.ServerError error_code = 2;
+ private com.yahoo.pulsar.common.api.proto.PulsarApi.ServerError errorCode_ = com.yahoo.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError;
+ public boolean hasErrorCode() {
+ return ((bitField0_ & 0x00000002) == 0x00000002);
+ }
+ public com.yahoo.pulsar.common.api.proto.PulsarApi.ServerError getErrorCode() {
+ return errorCode_;
+ }
+ public Builder setErrorCode(com.yahoo.pulsar.common.api.proto.PulsarApi.ServerError value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ bitField0_ |= 0x00000002;
+ errorCode_ = value;
+
+ return this;
+ }
+ public Builder clearErrorCode() {
+ bitField0_ = (bitField0_ & ~0x00000002);
+ errorCode_ = com.yahoo.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError;
+
+ return this;
+ }
+
+ // optional string error_message = 3;
+ private java.lang.Object errorMessage_ = "";
+ public boolean hasErrorMessage() {
+ return ((bitField0_ & 0x00000004) == 0x00000004);
+ }
+ public String getErrorMessage() {
+ java.lang.Object ref = errorMessage_;
+ if (!(ref instanceof String)) {
+ String s = ((com.google.protobuf.ByteString) ref).toStringUtf8();
+ errorMessage_ = s;
+ return s;
+ } else {
+ return (String) ref;
+ }
+ }
+ public Builder setErrorMessage(String value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ bitField0_ |= 0x00000004;
+ errorMessage_ = value;
+
+ return this;
+ }
+ public Builder clearErrorMessage() {
+ bitField0_ = (bitField0_ & ~0x00000004);
+ errorMessage_ = getDefaultInstance().getErrorMessage();
+
+ return this;
+ }
+ void setErrorMessage(com.google.protobuf.ByteString value) {
+ bitField0_ |= 0x00000004;
+ errorMessage_ = value;
+
+ }
+
+ // optional double msgRateOut = 4;
+ private double msgRateOut_ ;
+ public boolean hasMsgRateOut() {
+ return ((bitField0_ & 0x00000008) == 0x00000008);
+ }
+ public double getMsgRateOut() {
+ return msgRateOut_;
+ }
+ public Builder setMsgRateOut(double value) {
+ bitField0_ |= 0x00000008;
+ msgRateOut_ = value;
+
+ return this;
+ }
+ public Builder clearMsgRateOut() {
+ bitField0_ = (bitField0_ & ~0x00000008);
+ msgRateOut_ = 0D;
+
+ return this;
+ }
+
+ // optional double msgThroughputOut = 5;
+ private double msgThroughputOut_ ;
+ public boolean hasMsgThroughputOut() {
+ return ((bitField0_ & 0x00000010) == 0x00000010);
+ }
+ public double getMsgThroughputOut() {
+ return msgThroughputOut_;
+ }
+ public Builder setMsgThroughputOut(double value) {
+ bitField0_ |= 0x00000010;
+ msgThroughputOut_ = value;
+
+ return this;
+ }
+ public Builder clearMsgThroughputOut() {
+ bitField0_ = (bitField0_ & ~0x00000010);
+ msgThroughputOut_ = 0D;
+
+ return this;
+ }
+
+ // optional double msgRateRedeliver = 6;
+ private double msgRateRedeliver_ ;
+ public boolean hasMsgRateRedeliver() {
+ return ((bitField0_ & 0x00000020) == 0x00000020);
+ }
+ public double getMsgRateRedeliver() {
+ return msgRateRedeliver_;
+ }
+ public Builder setMsgRateRedeliver(double value) {
+ bitField0_ |= 0x00000020;
+ msgRateRedeliver_ = value;
+
+ return this;
+ }
+ public Builder clearMsgRateRedeliver() {
+ bitField0_ = (bitField0_ & ~0x00000020);
+ msgRateRedeliver_ = 0D;
+
+ return this;
+ }
+
+ // optional string consumerName = 7;
+ private java.lang.Object consumerName_ = "";
+ public boolean hasConsumerName() {
+ return ((bitField0_ & 0x00000040) == 0x00000040);
+ }
+ public String getConsumerName() {
+ java.lang.Object ref = consumerName_;
+ if (!(ref instanceof String)) {
+ String s = ((com.google.protobuf.ByteString) ref).toStringUtf8();
+ consumerName_ = s;
+ return s;
+ } else {
+ return (String) ref;
+ }
+ }
+ public Builder setConsumerName(String value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ bitField0_ |= 0x00000040;
+ consumerName_ = value;
+
+ return this;
+ }
+ public Builder clearConsumerName() {
+ bitField0_ = (bitField0_ & ~0x00000040);
+ consumerName_ = getDefaultInstance().getConsumerName();
+
+ return this;
+ }
+ void setConsumerName(com.google.protobuf.ByteString value) {
+ bitField0_ |= 0x00000040;
+ consumerName_ = value;
+
+ }
+
+ // optional uint64 availablePermits = 8;
+ private long availablePermits_ ;
+ public boolean hasAvailablePermits() {
+ return ((bitField0_ & 0x00000080) == 0x00000080);
+ }
+ public long getAvailablePermits() {
+ return availablePermits_;
+ }
+ public Builder setAvailablePermits(long value) {
+ bitField0_ |= 0x00000080;
+ availablePermits_ = value;
+
+ return this;
+ }
+ public Builder clearAvailablePermits() {
+ bitField0_ = (bitField0_ & ~0x00000080);
+ availablePermits_ = 0L;
+
+ return this;
+ }
+
+ // optional uint64 unackedMessages = 9;
+ private long unackedMessages_ ;
+ public boolean hasUnackedMessages() {
+ return ((bitField0_ & 0x00000100) == 0x00000100);
+ }
+ public long getUnackedMessages() {
+ return unackedMessages_;
+ }
+ public Builder setUnackedMessages(long value) {
+ bitField0_ |= 0x00000100;
+ unackedMessages_ = value;
+
+ return this;
+ }
+ public Builder clearUnackedMessages() {
+ bitField0_ = (bitField0_ & ~0x00000100);
+ unackedMessages_ = 0L;
+
+ return this;
+ }
+
+ // optional bool blockedConsumerOnUnackedMsgs = 10;
+ private boolean blockedConsumerOnUnackedMsgs_ ;
+ public boolean hasBlockedConsumerOnUnackedMsgs() {
+ return ((bitField0_ & 0x00000200) == 0x00000200);
+ }
+ public boolean getBlockedConsumerOnUnackedMsgs() {
+ return blockedConsumerOnUnackedMsgs_;
+ }
+ public Builder setBlockedConsumerOnUnackedMsgs(boolean value) {
+ bitField0_ |= 0x00000200;
+ blockedConsumerOnUnackedMsgs_ = value;
+
+ return this;
+ }
+ public Builder clearBlockedConsumerOnUnackedMsgs() {
+ bitField0_ = (bitField0_ & ~0x00000200);
+ blockedConsumerOnUnackedMsgs_ = false;
+
+ return this;
+ }
+
+ // optional string address = 11;
+ private java.lang.Object address_ = "";
+ public boolean hasAddress() {
+ return ((bitField0_ & 0x00000400) == 0x00000400);
+ }
+ public String getAddress() {
+ java.lang.Object ref = address_;
+ if (!(ref instanceof String)) {
+ String s = ((com.google.protobuf.ByteString) ref).toStringUtf8();
+ address_ = s;
+ return s;
+ } else {
+ return (String) ref;
+ }
+ }
+ public Builder setAddress(String value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ bitField0_ |= 0x00000400;
+ address_ = value;
+
+ return this;
+ }
+ public Builder clearAddress() {
+ bitField0_ = (bitField0_ & ~0x00000400);
+ address_ = getDefaultInstance().getAddress();
+
+ return this;
+ }
+ void setAddress(com.google.protobuf.ByteString value) {
+ bitField0_ |= 0x00000400;
+ address_ = value;
+
+ }
+
+ // optional string connectedSince = 12;
+ private java.lang.Object connectedSince_ = "";
+ public boolean hasConnectedSince() {
+ return ((bitField0_ & 0x00000800) == 0x00000800);
+ }
+ public String getConnectedSince() {
+ java.lang.Object ref = connectedSince_;
+ if (!(ref instanceof String)) {
+ String s = ((com.google.protobuf.ByteString) ref).toStringUtf8();
+ connectedSince_ = s;
+ return s;
+ } else {
+ return (String) ref;
+ }
+ }
+ public Builder setConnectedSince(String value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ bitField0_ |= 0x00000800;
+ connectedSince_ = value;
+
+ return this;
+ }
+ public Builder clearConnectedSince() {
+ bitField0_ = (bitField0_ & ~0x00000800);
+ connectedSince_ = getDefaultInstance().getConnectedSince();
+
+ return this;
+ }
+ void setConnectedSince(com.google.protobuf.ByteString value) {
+ bitField0_ |= 0x00000800;
+ connectedSince_ = value;
+
+ }
+
+ // optional string type = 13;
+ private java.lang.Object type_ = "";
+ public boolean hasType() {
+ return ((bitField0_ & 0x00001000) == 0x00001000);
+ }
+ public String getType() {
+ java.lang.Object ref = type_;
+ if (!(ref instanceof String)) {
+ String s = ((com.google.protobuf.ByteString) ref).toStringUtf8();
+ type_ = s;
+ return s;
+ } else {
+ return (String) ref;
+ }
+ }
+ public Builder setType(String value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ bitField0_ |= 0x00001000;
+ type_ = value;
+
+ return this;
+ }
+ public Builder clearType() {
+ bitField0_ = (bitField0_ & ~0x00001000);
+ type_ = getDefaultInstance().getType();
+
+ return this;
+ }
+ void setType(com.google.protobuf.ByteString value) {
+ bitField0_ |= 0x00001000;
+ type_ = value;
+
+ }
+
+ // optional double msgRateExpired = 14;
+ private double msgRateExpired_ ;
+ public boolean hasMsgRateExpired() {
+ return ((bitField0_ & 0x00002000) == 0x00002000);
+ }
+ public double getMsgRateExpired() {
+ return msgRateExpired_;
+ }
+ public Builder setMsgRateExpired(double value) {
+ bitField0_ |= 0x00002000;
+ msgRateExpired_ = value;
+
+ return this;
+ }
+ public Builder clearMsgRateExpired() {
+ bitField0_ = (bitField0_ & ~0x00002000);
+ msgRateExpired_ = 0D;
+
+ return this;
+ }
+
+ // optional uint64 msgBacklog = 15;
+ private long msgBacklog_ ;
+ public boolean hasMsgBacklog() {
+ return ((bitField0_ & 0x00004000) == 0x00004000);
+ }
+ public long getMsgBacklog() {
+ return msgBacklog_;
+ }
+ public Builder setMsgBacklog(long value) {
+ bitField0_ |= 0x00004000;
+ msgBacklog_ = value;
+
+ return this;
+ }
+ public Builder clearMsgBacklog() {
+ bitField0_ = (bitField0_ & ~0x00004000);
+ msgBacklog_ = 0L;
+
+ return this;
+ }
+
+ // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandConsumerStatsResponse)
+ }
+
+ static {
+ defaultInstance = new CommandConsumerStatsResponse(true);
+ defaultInstance.initFields();
+ }
+
+ // @@protoc_insertion_point(class_scope:pulsar.proto.CommandConsumerStatsResponse)
+ }
+
public interface BaseCommandOrBuilder
extends com.google.protobuf.MessageLiteOrBuilder {
@@ -14342,6 +16269,14 @@ public interface BaseCommandOrBuilder
// optional .pulsar.proto.CommandLookupTopicResponse lookupTopicResponse = 24;
boolean hasLookupTopicResponse();
com.yahoo.pulsar.common.api.proto.PulsarApi.CommandLookupTopicResponse getLookupTopicResponse();
+
+ // optional .pulsar.proto.CommandConsumerStats consumerStats = 25;
+ boolean hasConsumerStats();
+ com.yahoo.pulsar.common.api.proto.PulsarApi.CommandConsumerStats getConsumerStats();
+
+ // optional .pulsar.proto.CommandConsumerStatsResponse consumerStatsResponse = 26;
+ boolean hasConsumerStatsResponse();
+ com.yahoo.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse getConsumerStatsResponse();
}
public static final class BaseCommand extends
com.google.protobuf.GeneratedMessageLite
@@ -14402,6 +16337,8 @@ public enum Type
PARTITIONED_METADATA_RESPONSE(20, 22),
LOOKUP(21, 23),
LOOKUP_RESPONSE(22, 24),
+ CONSUMER_STATS(23, 25),
+ CONSUMER_STATS_RESPONSE(24, 26),
;
public static final int CONNECT_VALUE = 2;
@@ -14427,6 +16364,8 @@ public enum Type
public static final int PARTITIONED_METADATA_RESPONSE_VALUE = 22;
public static final int LOOKUP_VALUE = 23;
public static final int LOOKUP_RESPONSE_VALUE = 24;
+ public static final int CONSUMER_STATS_VALUE = 25;
+ public static final int CONSUMER_STATS_RESPONSE_VALUE = 26;
public final int getNumber() { return value; }
@@ -14456,6 +16395,8 @@ public static Type valueOf(int value) {
case 22: return PARTITIONED_METADATA_RESPONSE;
case 23: return LOOKUP;
case 24: return LOOKUP_RESPONSE;
+ case 25: return CONSUMER_STATS;
+ case 26: return CONSUMER_STATS_RESPONSE;
default: return null;
}
}
@@ -14722,6 +16663,26 @@ public com.yahoo.pulsar.common.api.proto.PulsarApi.CommandLookupTopicResponse ge
return lookupTopicResponse_;
}
+ // optional .pulsar.proto.CommandConsumerStats consumerStats = 25;
+ public static final int CONSUMERSTATS_FIELD_NUMBER = 25;
+ private com.yahoo.pulsar.common.api.proto.PulsarApi.CommandConsumerStats consumerStats_;
+ public boolean hasConsumerStats() {
+ return ((bitField0_ & 0x01000000) == 0x01000000);
+ }
+ public com.yahoo.pulsar.common.api.proto.PulsarApi.CommandConsumerStats getConsumerStats() {
+ return consumerStats_;
+ }
+
+ // optional .pulsar.proto.CommandConsumerStatsResponse consumerStatsResponse = 26;
+ public static final int CONSUMERSTATSRESPONSE_FIELD_NUMBER = 26;
+ private com.yahoo.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse consumerStatsResponse_;
+ public boolean hasConsumerStatsResponse() {
+ return ((bitField0_ & 0x02000000) == 0x02000000);
+ }
+ public com.yahoo.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse getConsumerStatsResponse() {
+ return consumerStatsResponse_;
+ }
+
private void initFields() {
type_ = com.yahoo.pulsar.common.api.proto.PulsarApi.BaseCommand.Type.CONNECT;
connect_ = com.yahoo.pulsar.common.api.proto.PulsarApi.CommandConnect.getDefaultInstance();
@@ -14747,6 +16708,8 @@ private void initFields() {
partitionMetadataResponse_ = com.yahoo.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadataResponse.getDefaultInstance();
lookupTopic_ = com.yahoo.pulsar.common.api.proto.PulsarApi.CommandLookupTopic.getDefaultInstance();
lookupTopicResponse_ = com.yahoo.pulsar.common.api.proto.PulsarApi.CommandLookupTopicResponse.getDefaultInstance();
+ consumerStats_ = com.yahoo.pulsar.common.api.proto.PulsarApi.CommandConsumerStats.getDefaultInstance();
+ consumerStatsResponse_ = com.yahoo.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse.getDefaultInstance();
}
private byte memoizedIsInitialized = -1;
public final boolean isInitialized() {
@@ -14883,6 +16846,18 @@ public final boolean isInitialized() {
return false;
}
}
+ if (hasConsumerStats()) {
+ if (!getConsumerStats().isInitialized()) {
+ memoizedIsInitialized = 0;
+ return false;
+ }
+ }
+ if (hasConsumerStatsResponse()) {
+ if (!getConsumerStatsResponse().isInitialized()) {
+ memoizedIsInitialized = 0;
+ return false;
+ }
+ }
memoizedIsInitialized = 1;
return true;
}
@@ -14967,6 +16942,12 @@ public void writeTo(com.yahoo.pulsar.common.util.protobuf.ByteBufCodedOutputStre
if (((bitField0_ & 0x00800000) == 0x00800000)) {
output.writeMessage(24, lookupTopicResponse_);
}
+ if (((bitField0_ & 0x01000000) == 0x01000000)) {
+ output.writeMessage(25, consumerStats_);
+ }
+ if (((bitField0_ & 0x02000000) == 0x02000000)) {
+ output.writeMessage(26, consumerStatsResponse_);
+ }
}
private int memoizedSerializedSize = -1;
@@ -15071,6 +17052,14 @@ public int getSerializedSize() {
size += com.google.protobuf.CodedOutputStream
.computeMessageSize(24, lookupTopicResponse_);
}
+ if (((bitField0_ & 0x01000000) == 0x01000000)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeMessageSize(25, consumerStats_);
+ }
+ if (((bitField0_ & 0x02000000) == 0x02000000)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeMessageSize(26, consumerStatsResponse_);
+ }
memoizedSerializedSize = size;
return size;
}
@@ -15232,6 +17221,10 @@ public Builder clear() {
bitField0_ = (bitField0_ & ~0x00400000);
lookupTopicResponse_ = com.yahoo.pulsar.common.api.proto.PulsarApi.CommandLookupTopicResponse.getDefaultInstance();
bitField0_ = (bitField0_ & ~0x00800000);
+ consumerStats_ = com.yahoo.pulsar.common.api.proto.PulsarApi.CommandConsumerStats.getDefaultInstance();
+ bitField0_ = (bitField0_ & ~0x01000000);
+ consumerStatsResponse_ = com.yahoo.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse.getDefaultInstance();
+ bitField0_ = (bitField0_ & ~0x02000000);
return this;
}
@@ -15361,6 +17354,14 @@ public com.yahoo.pulsar.common.api.proto.PulsarApi.BaseCommand buildPartial() {
to_bitField0_ |= 0x00800000;
}
result.lookupTopicResponse_ = lookupTopicResponse_;
+ if (((from_bitField0_ & 0x01000000) == 0x01000000)) {
+ to_bitField0_ |= 0x01000000;
+ }
+ result.consumerStats_ = consumerStats_;
+ if (((from_bitField0_ & 0x02000000) == 0x02000000)) {
+ to_bitField0_ |= 0x02000000;
+ }
+ result.consumerStatsResponse_ = consumerStatsResponse_;
result.bitField0_ = to_bitField0_;
return result;
}
@@ -15439,6 +17440,12 @@ public Builder mergeFrom(com.yahoo.pulsar.common.api.proto.PulsarApi.BaseCommand
if (other.hasLookupTopicResponse()) {
mergeLookupTopicResponse(other.getLookupTopicResponse());
}
+ if (other.hasConsumerStats()) {
+ mergeConsumerStats(other.getConsumerStats());
+ }
+ if (other.hasConsumerStatsResponse()) {
+ mergeConsumerStatsResponse(other.getConsumerStatsResponse());
+ }
return this;
}
@@ -15573,6 +17580,18 @@ public final boolean isInitialized() {
return false;
}
}
+ if (hasConsumerStats()) {
+ if (!getConsumerStats().isInitialized()) {
+
+ return false;
+ }
+ }
+ if (hasConsumerStatsResponse()) {
+ if (!getConsumerStatsResponse().isInitialized()) {
+
+ return false;
+ }
+ }
return true;
}
@@ -15837,6 +17856,26 @@ public Builder mergeFrom(
subBuilder.recycle();
break;
}
+ case 202: {
+ com.yahoo.pulsar.common.api.proto.PulsarApi.CommandConsumerStats.Builder subBuilder = com.yahoo.pulsar.common.api.proto.PulsarApi.CommandConsumerStats.newBuilder();
+ if (hasConsumerStats()) {
+ subBuilder.mergeFrom(getConsumerStats());
+ }
+ input.readMessage(subBuilder, extensionRegistry);
+ setConsumerStats(subBuilder.buildPartial());
+ subBuilder.recycle();
+ break;
+ }
+ case 210: {
+ com.yahoo.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse.Builder subBuilder = com.yahoo.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse.newBuilder();
+ if (hasConsumerStatsResponse()) {
+ subBuilder.mergeFrom(getConsumerStatsResponse());
+ }
+ input.readMessage(subBuilder, extensionRegistry);
+ setConsumerStatsResponse(subBuilder.buildPartial());
+ subBuilder.recycle();
+ break;
+ }
}
}
}
@@ -16856,6 +18895,92 @@ public Builder clearLookupTopicResponse() {
return this;
}
+ // optional .pulsar.proto.CommandConsumerStats consumerStats = 25;
+ private com.yahoo.pulsar.common.api.proto.PulsarApi.CommandConsumerStats consumerStats_ = com.yahoo.pulsar.common.api.proto.PulsarApi.CommandConsumerStats.getDefaultInstance();
+ public boolean hasConsumerStats() {
+ return ((bitField0_ & 0x01000000) == 0x01000000);
+ }
+ public com.yahoo.pulsar.common.api.proto.PulsarApi.CommandConsumerStats getConsumerStats() {
+ return consumerStats_;
+ }
+ public Builder setConsumerStats(com.yahoo.pulsar.common.api.proto.PulsarApi.CommandConsumerStats value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ consumerStats_ = value;
+
+ bitField0_ |= 0x01000000;
+ return this;
+ }
+ public Builder setConsumerStats(
+ com.yahoo.pulsar.common.api.proto.PulsarApi.CommandConsumerStats.Builder builderForValue) {
+ consumerStats_ = builderForValue.build();
+
+ bitField0_ |= 0x01000000;
+ return this;
+ }
+ public Builder mergeConsumerStats(com.yahoo.pulsar.common.api.proto.PulsarApi.CommandConsumerStats value) {
+ if (((bitField0_ & 0x01000000) == 0x01000000) &&
+ consumerStats_ != com.yahoo.pulsar.common.api.proto.PulsarApi.CommandConsumerStats.getDefaultInstance()) {
+ consumerStats_ =
+ com.yahoo.pulsar.common.api.proto.PulsarApi.CommandConsumerStats.newBuilder(consumerStats_).mergeFrom(value).buildPartial();
+ } else {
+ consumerStats_ = value;
+ }
+
+ bitField0_ |= 0x01000000;
+ return this;
+ }
+ public Builder clearConsumerStats() {
+ consumerStats_ = com.yahoo.pulsar.common.api.proto.PulsarApi.CommandConsumerStats.getDefaultInstance();
+
+ bitField0_ = (bitField0_ & ~0x01000000);
+ return this;
+ }
+
+ // optional .pulsar.proto.CommandConsumerStatsResponse consumerStatsResponse = 26;
+ private com.yahoo.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse consumerStatsResponse_ = com.yahoo.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse.getDefaultInstance();
+ public boolean hasConsumerStatsResponse() {
+ return ((bitField0_ & 0x02000000) == 0x02000000);
+ }
+ public com.yahoo.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse getConsumerStatsResponse() {
+ return consumerStatsResponse_;
+ }
+ public Builder setConsumerStatsResponse(com.yahoo.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ consumerStatsResponse_ = value;
+
+ bitField0_ |= 0x02000000;
+ return this;
+ }
+ public Builder setConsumerStatsResponse(
+ com.yahoo.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse.Builder builderForValue) {
+ consumerStatsResponse_ = builderForValue.build();
+
+ bitField0_ |= 0x02000000;
+ return this;
+ }
+ public Builder mergeConsumerStatsResponse(com.yahoo.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse value) {
+ if (((bitField0_ & 0x02000000) == 0x02000000) &&
+ consumerStatsResponse_ != com.yahoo.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse.getDefaultInstance()) {
+ consumerStatsResponse_ =
+ com.yahoo.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse.newBuilder(consumerStatsResponse_).mergeFrom(value).buildPartial();
+ } else {
+ consumerStatsResponse_ = value;
+ }
+
+ bitField0_ |= 0x02000000;
+ return this;
+ }
+ public Builder clearConsumerStatsResponse() {
+ consumerStatsResponse_ = com.yahoo.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse.getDefaultInstance();
+
+ bitField0_ = (bitField0_ & ~0x02000000);
+ return this;
+ }
+
// @@protoc_insertion_point(builder_scope:pulsar.proto.BaseCommand)
}
diff --git a/pulsar-common/src/main/java/com/yahoo/pulsar/common/util/protobuf/ByteBufCodedOutputStream.java b/pulsar-common/src/main/java/com/yahoo/pulsar/common/util/protobuf/ByteBufCodedOutputStream.java
index 8086a7dffef3e..5a9e9aec376fe 100644
--- a/pulsar-common/src/main/java/com/yahoo/pulsar/common/util/protobuf/ByteBufCodedOutputStream.java
+++ b/pulsar-common/src/main/java/com/yahoo/pulsar/common/util/protobuf/ByteBufCodedOutputStream.java
@@ -242,4 +242,11 @@ public void writeMessageNoTag(final ByteBufGeneratedMessage value) throws IOExce
static int makeTag(final int fieldNumber, final int wireType) {
return (fieldNumber << TAG_TYPE_BITS) | wireType;
}
+
+ /** Write an double field, including tag, to the stream. */
+ public void writeDouble(final int fieldNumber, double value) throws IOException {
+ writeTag(fieldNumber, WireFormat.WIRETYPE_FIXED64);
+ buf.order(ByteOrder.LITTLE_ENDIAN).writeDouble(value);
+ }
+
}
diff --git a/pulsar-common/src/main/proto/PulsarApi.proto b/pulsar-common/src/main/proto/PulsarApi.proto
index 8e4392e6928c1..07da8a7878fc9 100644
--- a/pulsar-common/src/main/proto/PulsarApi.proto
+++ b/pulsar-common/src/main/proto/PulsarApi.proto
@@ -77,7 +77,10 @@ enum ServerError {
ProducerBlockedQuotaExceededError = 7; // Unable to create producer because backlog quota exceeded
ProducerBlockedQuotaExceededException = 8; // Exception while creating producer because quota exceeded
ChecksumError = 9; // Error while verifying message checksum
- UnsupportedVersionError = 10; // Error when an older client/version doesn't support a required feature
+ UnsupportedVersionError = 10; // Error when an older client/version doesn't support a required feature
+ TopicNotFound = 11; // Topic not found
+ SubscriptionNotFound = 12; // Subscription not found
+ ConsumerNotFound = 13; // Consumer not found
}
enum AuthMethod {
@@ -96,6 +99,7 @@ enum ProtocolVersion {
v4 = 4; // Added batch message support
v5 = 5; // Added disconnect client w/o closing connection
v6 = 6; // Added checksum computation for metadata + payload
+ v7 = 7; // Added Lookup and GetStats Command
}
message CommandConnect {
@@ -117,7 +121,6 @@ message CommandSubscribe {
Shared = 1;
Failover = 2;
}
-
required string topic = 1;
required string subscription = 2;
required SubType subType = 3;
@@ -279,6 +282,55 @@ message CommandPing {
message CommandPong {
}
+message CommandConsumerStats {
+ required uint64 request_id = 1;
+ required string topic_name = 2;
+ required string subscription_name = 3;
+ required uint64 consumer_id = 4;
+}
+
+message CommandConsumerStatsResponse {
+ required uint64 request_id = 1;
+ optional ServerError error_code = 2;
+ optional string error_message = 3;
+
+ /// Total rate of messages delivered to the consumer. msg/s
+ optional double msgRateOut = 4;
+
+ /// Total throughput delivered to the consumer. bytes/s
+ optional double msgThroughputOut = 5;
+
+ /// Total rate of messages redelivered by this consumer. msg/s
+ optional double msgRateRedeliver = 6;
+
+ /// Name of the consumer
+ optional string consumerName = 7;
+
+ /// Number of available message permits for the consumer
+ optional uint64 availablePermits = 8;
+
+ /// Number of unacknowledged messages for the consumer
+ optional uint64 unackedMessages = 9;
+
+ /// Flag to verify if consumer is blocked due to reaching threshold of unacked messages
+ optional bool blockedConsumerOnUnackedMsgs = 10;
+
+ /// Address of this consumer
+ optional string address = 11;
+
+ /// Timestamp of connection
+ optional string connectedSince = 12;
+
+ /// Whether this subscription is Exclusive or Shared or Failover
+ optional string type = 13;
+
+ /// Total rate of messages expired on this subscription. msg/s
+ optional double msgRateExpired = 14;
+
+ /// Number of messages in the subscription backlog
+ optional uint64 msgBacklog = 15;
+}
+
message BaseCommand {
enum Type {
CONNECT = 2;
@@ -315,6 +367,10 @@ message BaseCommand {
LOOKUP = 23;
LOOKUP_RESPONSE = 24;
+
+ CONSUMER_STATS = 25;
+ CONSUMER_STATS_RESPONSE = 26;
+
}
required Type type = 1;
@@ -348,4 +404,7 @@ message BaseCommand {
optional CommandLookupTopic lookupTopic = 23;
optional CommandLookupTopicResponse lookupTopicResponse = 24;
+
+ optional CommandConsumerStats consumerStats = 25;
+ optional CommandConsumerStatsResponse consumerStatsResponse = 26;
}