From b3db513bcc1733ba4dd2bb9448da297b2ec7a5bf Mon Sep 17 00:00:00 2001 From: jai1 Date: Wed, 28 Jun 2017 15:37:00 -0700 Subject: [PATCH] CPP Client - Added Monitoring Stats (#485) --- .../include/pulsar/ClientConfiguration.h | 12 + .../include/pulsar/ProducerConfiguration.h | 1 + pulsar-client-cpp/lib/CMakeLists.txt | 2 +- pulsar-client-cpp/lib/ClientConfiguration.cc | 9 + .../lib/ClientConfigurationImpl.h | 5 +- pulsar-client-cpp/lib/ClientImpl.cc | 5 + pulsar-client-cpp/lib/ClientImpl.h | 2 + pulsar-client-cpp/lib/ConsumerImpl.cc | 43 +- pulsar-client-cpp/lib/ConsumerImpl.h | 10 + pulsar-client-cpp/lib/ProducerImpl.cc | 45 +- pulsar-client-cpp/lib/ProducerImpl.h | 9 +- pulsar-client-cpp/lib/Result.cc | 4 +- pulsar-client-cpp/lib/Utils.h | 9 + .../lib/stats/ConsumerStatsBase.h | 38 + .../lib/stats/ConsumerStatsDisabled.h | 35 + .../lib/stats/ConsumerStatsImpl.cc | 113 ++ .../lib/stats/ConsumerStatsImpl.h | 83 ++ .../lib/stats/ProducerStatsBase.h | 37 + .../lib/stats/ProducerStatsDisabled.h | 32 + .../lib/stats/ProducerStatsImpl.cc | 132 ++ .../lib/stats/ProducerStatsImpl.h | 118 ++ pulsar-client-cpp/log | 1202 +++++++++++++++++ pulsar-client-cpp/tests/BasicEndToEndTest.cc | 167 ++- pulsar-client-cpp/tests/BatchMessageTest.cc | 125 +- pulsar-client-cpp/tests/PulsarFriend.h | 21 + 25 files changed, 2176 insertions(+), 83 deletions(-) create mode 100644 pulsar-client-cpp/lib/stats/ConsumerStatsBase.h create mode 100644 pulsar-client-cpp/lib/stats/ConsumerStatsDisabled.h create mode 100644 pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc create mode 100644 pulsar-client-cpp/lib/stats/ConsumerStatsImpl.h create mode 100644 pulsar-client-cpp/lib/stats/ProducerStatsBase.h create mode 100644 pulsar-client-cpp/lib/stats/ProducerStatsDisabled.h create mode 100644 pulsar-client-cpp/lib/stats/ProducerStatsImpl.cc create mode 100644 pulsar-client-cpp/lib/stats/ProducerStatsImpl.h create mode 100644 pulsar-client-cpp/log diff --git a/pulsar-client-cpp/include/pulsar/ClientConfiguration.h b/pulsar-client-cpp/include/pulsar/ClientConfiguration.h index b385dcbdbd3f6..e8914a43cbf76 100644 --- a/pulsar-client-cpp/include/pulsar/ClientConfiguration.h +++ b/pulsar-client-cpp/include/pulsar/ClientConfiguration.h @@ -121,6 +121,18 @@ class ClientConfiguration { ClientConfiguration& setTlsAllowInsecureConnection(bool allowInsecure); bool isTlsAllowInsecureConnection() const; + + /* + * Initialize stats interval in seconds. Stats are printed and reset after every 'statsIntervalInSeconds'. + * Set to 0 in order to disable stats collection. + */ + ClientConfiguration& setStatsIntervalInSeconds(const unsigned int&); + + /* + * Get the stats interval set in the client. + */ + const unsigned int& getStatsIntervalInSeconds() const; + friend class ClientImpl; friend class PulsarWrapper; diff --git a/pulsar-client-cpp/include/pulsar/ProducerConfiguration.h b/pulsar-client-cpp/include/pulsar/ProducerConfiguration.h index 296b5f79ebac9..238d9807b0bb1 100644 --- a/pulsar-client-cpp/include/pulsar/ProducerConfiguration.h +++ b/pulsar-client-cpp/include/pulsar/ProducerConfiguration.h @@ -80,6 +80,7 @@ class ProducerConfiguration { ProducerConfiguration& setBatchingMaxPublishDelayMs( const unsigned long& batchingMaxPublishDelayMs); const unsigned long& getBatchingMaxPublishDelayMs() const; + friend class PulsarWrapper; private: diff --git a/pulsar-client-cpp/lib/CMakeLists.txt b/pulsar-client-cpp/lib/CMakeLists.txt index 16bf1cbfbdc68..a9bf83fe9c2b7 100644 --- a/pulsar-client-cpp/lib/CMakeLists.txt +++ b/pulsar-client-cpp/lib/CMakeLists.txt @@ -17,7 +17,7 @@ # under the License. # -file(GLOB PULSAR_SOURCES *.cc lz4/*.c checksum/*.cc) +file(GLOB PULSAR_SOURCES *.cc lz4/*.c checksum/*.cc stats/*.cc) execute_process(COMMAND cat ../pom.xml COMMAND xmllint --format - COMMAND sed "s/xmlns=\".*\"//g" COMMAND xmllint --stream --pattern /project/version --debug - COMMAND grep -A 2 "matches pattern" COMMAND grep text COMMAND sed "s/.* [0-9] //g" OUTPUT_STRIP_TRAILING_WHITESPACE OUTPUT_VARIABLE PV) set (CMAKE_CXX_FLAGS " ${CMAKE_CXX_FLAGS} -msse4.2 -mpclmul -D_PULSAR_VERSION_=\\\"${PV}\\\"") diff --git a/pulsar-client-cpp/lib/ClientConfiguration.cc b/pulsar-client-cpp/lib/ClientConfiguration.cc index ba91ffc25e224..d7f30ba1829d1 100644 --- a/pulsar-client-cpp/lib/ClientConfiguration.cc +++ b/pulsar-client-cpp/lib/ClientConfiguration.cc @@ -121,4 +121,13 @@ const std::string& ClientConfiguration::getLogConfFilePath() const { return impl_->logConfFilePath; } +ClientConfiguration& ClientConfiguration::setStatsIntervalInSeconds(const unsigned int& statsIntervalInSeconds) { + impl_->statsIntervalInSeconds = statsIntervalInSeconds; + return *this; +} + +const unsigned int& ClientConfiguration::getStatsIntervalInSeconds() const { + return impl_->statsIntervalInSeconds; +} + } diff --git a/pulsar-client-cpp/lib/ClientConfigurationImpl.h b/pulsar-client-cpp/lib/ClientConfigurationImpl.h index 46f028402cdf6..40b6a9b5e78a9 100644 --- a/pulsar-client-cpp/lib/ClientConfigurationImpl.h +++ b/pulsar-client-cpp/lib/ClientConfigurationImpl.h @@ -33,6 +33,7 @@ struct ClientConfigurationImpl { bool useTls; std::string tlsTrustCertsFilePath; bool tlsAllowInsecureConnection; + unsigned int statsIntervalInSeconds; ClientConfigurationImpl() : authenticationPtr(AuthFactory::Disabled()), ioThreads(1), operationTimeoutSeconds(30), @@ -40,7 +41,9 @@ struct ClientConfigurationImpl { concurrentLookupRequest(5000), logConfFilePath(), useTls(false), - tlsAllowInsecureConnection(true) {} + tlsAllowInsecureConnection(true), + statsIntervalInSeconds(600) { // 10 minutes + } }; } diff --git a/pulsar-client-cpp/lib/ClientImpl.cc b/pulsar-client-cpp/lib/ClientImpl.cc index 34c608bbc7223..2b339db5b6269 100644 --- a/pulsar-client-cpp/lib/ClientImpl.cc +++ b/pulsar-client-cpp/lib/ClientImpl.cc @@ -83,6 +83,7 @@ namespace pulsar { } ClientImpl::~ClientImpl() { + LOG_DEBUG("~ClientImpl"); shutdown(); } @@ -367,4 +368,8 @@ namespace pulsar { return requestIdGenerator_++; } + const ClientConfiguration& ClientImpl::getClientConfig() const { + return clientConfiguration_; + } + } /* namespace pulsar */ diff --git a/pulsar-client-cpp/lib/ClientImpl.h b/pulsar-client-cpp/lib/ClientImpl.h index 2f07254487915..7fe353b6bad1c 100644 --- a/pulsar-client-cpp/lib/ClientImpl.h +++ b/pulsar-client-cpp/lib/ClientImpl.h @@ -64,6 +64,8 @@ class ClientImpl : public boost::enable_shared_from_this { uint64_t newConsumerId(); uint64_t newRequestId(); + const ClientConfiguration& getClientConfig() const; + const ClientConfiguration& conf() const; ExecutorServiceProviderPtr getIOExecutorProvider(); ExecutorServiceProviderPtr getListenerExecutorProvider(); diff --git a/pulsar-client-cpp/lib/ConsumerImpl.cc b/pulsar-client-cpp/lib/ConsumerImpl.cc index f174a71bb4885..5a103a9fea43e 100644 --- a/pulsar-client-cpp/lib/ConsumerImpl.cc +++ b/pulsar-client-cpp/lib/ConsumerImpl.cc @@ -53,7 +53,8 @@ ConsumerImpl::ConsumerImpl(const ClientImplPtr client, const std::string& topic, consumerCreatedPromise_(), messageListenerRunning_(true), batchAcknowledgementTracker_(topic_, subscription, (long)consumerId_), - brokerConsumerStats_() { + brokerConsumerStats_(), + consumerStatsBasePtr_() { std::stringstream consumerStrStream; consumerStrStream << "[" << topic_ << ", " << subscription_ << ", " << consumerId_ << "] "; consumerStr_ = consumerStrStream.str(); @@ -67,6 +68,15 @@ ConsumerImpl::ConsumerImpl(const ClientImplPtr client, const std::string& topic, } else { listenerExecutor_ = client->getListenerExecutorProvider()->get(); } + + unsigned int statsIntervalInSeconds = client->getClientConfig().getStatsIntervalInSeconds(); + if (statsIntervalInSeconds) { + consumerStatsBasePtr_ = boost::make_shared( + consumerStr_, client->getIOExecutorProvider()->get()->createDeadlineTimer(), + statsIntervalInSeconds); + } else { + consumerStatsBasePtr_ = boost::make_shared(); + } } ConsumerImpl::~ConsumerImpl() { @@ -357,6 +367,7 @@ void ConsumerImpl::internalListener() { return; } try { + consumerStatsBasePtr_->receivedMessage(msg, ResultOk); messageListener_(Consumer(shared_from_this()), msg); } catch (const std::exception& e) { LOG_ERROR(getName() << "Exception thrown from listener" << e.what()); @@ -407,6 +418,12 @@ Result ConsumerImpl::fetchSingleMessageFromBroker(Message& msg) { } Result ConsumerImpl::receive(Message& msg) { + Result res = receiveHelper(msg); + consumerStatsBasePtr_->receivedMessage(msg, res); + return res; +} + +Result ConsumerImpl::receiveHelper(Message& msg) { { Lock lock(mutex_); if (state_ != Ready) { @@ -429,6 +446,12 @@ Result ConsumerImpl::receive(Message& msg) { } Result ConsumerImpl::receive(Message& msg, int timeout) { + Result res = receiveHelper(msg, timeout); + consumerStatsBasePtr_->receivedMessage(msg, res); + return res; +} + +Result ConsumerImpl::receiveHelper(Message& msg, int timeout) { if (config_.getReceiverQueueSize() == 0) { LOG_WARN(getName() << "Can't use this function if the queue size is 0"); return ResultInvalidConfiguration; @@ -496,27 +519,35 @@ inline proto::CommandSubscribe_SubType ConsumerImpl::getSubType() { } } + +void ConsumerImpl::statsCallback(Result res, ResultCallback callback, proto::CommandAck_AckType ackType) { + consumerStatsBasePtr_->messageAcknowledged(res, ackType); + callback(res); +} + void ConsumerImpl::acknowledgeAsync(const MessageId& msgId, ResultCallback callback) { + ResultCallback cb = boost::bind(&ConsumerImpl::statsCallback, this, _1, callback, proto::CommandAck_AckType_Individual); const BatchMessageId& batchMsgId = (const BatchMessageId&)msgId; if(batchMsgId.batchIndex_ != -1 && !batchAcknowledgementTracker_.isBatchReady(batchMsgId, proto::CommandAck_AckType_Individual)) { - callback(ResultOk); + cb(ResultOk); return; } - doAcknowledge(batchMsgId, proto::CommandAck_AckType_Individual, callback); + doAcknowledge(batchMsgId, proto::CommandAck_AckType_Individual, cb); } void ConsumerImpl::acknowledgeCumulativeAsync(const MessageId& mId, ResultCallback callback) { + ResultCallback cb = boost::bind(&ConsumerImpl::statsCallback, this, _1, callback, proto::CommandAck_AckType_Cumulative); const BatchMessageId& msgId = (const BatchMessageId&) mId; if(msgId.batchIndex_ != -1 && !batchAcknowledgementTracker_.isBatchReady(msgId, proto::CommandAck_AckType_Cumulative)) { BatchMessageId messageId = batchAcknowledgementTracker_.getGreatestCumulativeAckReady(msgId); if(messageId == BatchMessageId()) { // nothing to ack - callback(ResultOk); + cb(ResultOk); } else { - doAcknowledge(messageId, proto::CommandAck_AckType_Cumulative, callback); + doAcknowledge(messageId, proto::CommandAck_AckType_Cumulative, cb); } } else { - doAcknowledge(msgId, proto::CommandAck_AckType_Cumulative, callback); + doAcknowledge(msgId, proto::CommandAck_AckType_Cumulative, cb); } } diff --git a/pulsar-client-cpp/lib/ConsumerImpl.h b/pulsar-client-cpp/lib/ConsumerImpl.h index 76869d94acd31..e501896a25d8e 100644 --- a/pulsar-client-cpp/lib/ConsumerImpl.h +++ b/pulsar-client-cpp/lib/ConsumerImpl.h @@ -39,6 +39,8 @@ #include "BatchAcknowledgementTracker.h" #include #include +#include +#include using namespace pulsar; @@ -107,6 +109,7 @@ enum ConsumerTopicType { } virtual const std::string& getName() const; virtual int getNumOfPrefetchedMessages() const ; + ConsumerStatsBasePtr consumerStatsBasePtr_; private: bool waitingForZeroQueueSizeMessage; bool uncompressMessageIfNeeded(const ClientConnectionPtr& cnx, const proto::CommandMessage& msg, @@ -119,6 +122,11 @@ enum ConsumerTopicType { unsigned int receiveIndividualMessagesFromBatch(Message &batchedMessage); void brokerConsumerStatsListener(Result, BrokerConsumerStatsImpl, BrokerConsumerStatsCallback); + // TODO - Convert these functions to lambda when we move to C++11 + Result receiveHelper(Message& msg); + Result receiveHelper(Message& msg, int timeout); + void statsCallback(Result, ResultCallback, proto::CommandAck_AckType); + boost::mutex mutexForReceiveWithZeroQueueSize; const ConsumerConfiguration config_; const std::string subscription_; @@ -139,6 +147,8 @@ enum ConsumerTopicType { UnAckedMessageTrackerScopedPtr unAckedMessageTrackerPtr_; BatchAcknowledgementTracker batchAcknowledgementTracker_; BrokerConsumerStatsImpl brokerConsumerStats_; + + friend class PulsarFriend; }; } /* namespace pulsar */ diff --git a/pulsar-client-cpp/lib/ProducerImpl.cc b/pulsar-client-cpp/lib/ProducerImpl.cc index 5f0d1550e6194..f64278c8644b8 100644 --- a/pulsar-client-cpp/lib/ProducerImpl.cc +++ b/pulsar-client-cpp/lib/ProducerImpl.cc @@ -60,12 +60,17 @@ ProducerImpl::ProducerImpl(ClientImplPtr client, const std::string& topic, LOG_DEBUG( "ProducerName - " << producerName_ << " Created producer on topic " << topic_ << " id: " << producerId_); // boost::ref is used to drop the constantness constraint of make_shared - if(conf_.getBatchingEnabled()) { + if (conf_.getBatchingEnabled()) { batchMessageContainer = boost::make_shared(boost::ref(*this)); } - numOfMsgPublished = 0; - numOfSendAsyncCalls = 0; - numOfMsgAckSuccessfully = 0; + + unsigned int statsIntervalInSeconds = client->getClientConfig().getStatsIntervalInSeconds(); + if (statsIntervalInSeconds) { + producerStatsBasePtr_ = boost::make_shared(producerStr_, executor_->createDeadlineTimer(), + statsIntervalInSeconds); + } else { + producerStatsBasePtr_ = boost::make_shared(); + } } ProducerImpl::~ProducerImpl() { @@ -234,7 +239,14 @@ void ProducerImpl::setMessageMetadata(const Message &msg, const uint64_t& sequen } } +void ProducerImpl::statsCallBackHandler(Result res, const Message& msg, SendCallback callback, boost::posix_time::ptime publishTime) { + producerStatsBasePtr_->messageReceived(res, publishTime); + callback(res, msg); +} + void ProducerImpl::sendAsync(const Message& msg, SendCallback callback) { + producerStatsBasePtr_->messageSent(msg); + SendCallback cb = boost::bind(&ProducerImpl::statsCallBackHandler, this, _1, _2, callback, boost::posix_time::microsec_clock::universal_time()); if (msg.getLength() > Commands::MaxMessageSize) { callback(ResultMessageTooBig, msg); return; @@ -258,13 +270,12 @@ void ProducerImpl::sendAsync(const Message& msg, SendCallback callback) { } Lock lock(mutex_); - numOfSendAsyncCalls++; if (state_ != Ready) { lock.unlock(); if (conf_.getBlockIfQueueFull()) { pendingMessagesQueue_.release(1); } - callback(ResultAlreadyClosed, msg); + cb(ResultAlreadyClosed, msg); return; } @@ -274,7 +285,7 @@ void ProducerImpl::sendAsync(const Message& msg, SendCallback callback) { if (conf_.getBlockIfQueueFull()) { pendingMessagesQueue_.release(1); } - callback(ResultInvalidMessage, msg); + cb(ResultInvalidMessage, msg); return; } @@ -289,17 +300,17 @@ void ProducerImpl::sendAsync(const Message& msg, SendCallback callback) { batchMessageContainer->sendMessage(); } lock.unlock(); - callback(ResultProducerQueueIsFull, msg); + cb(ResultProducerQueueIsFull, msg); return; } // If we reach this point then you have a reserved spot on the queue if (batchMessageContainer) { // Batching is enabled - batchMessageContainer->add(msg, callback); + batchMessageContainer->add(msg, cb); return; } - sendMessage(msg, callback); + sendMessage(msg, cb); } // Precondition - @@ -314,7 +325,6 @@ void ProducerImpl::sendMessage(const Message& msg, SendCallback callback) { LOG_DEBUG("Inserting data to pendingMessagesQueue_"); pendingMessagesQueue_.push(op, true); - numOfMsgPublished++; LOG_DEBUG("Completed Inserting data to pendingMessagesQueue_"); ClientConnectionPtr cnx = getCnx().lock(); @@ -340,14 +350,11 @@ void ProducerImpl::batchMessageTimeoutHandler(const boost::system::error_code& e void ProducerImpl::printStats() { if (batchMessageContainer) { - LOG_INFO("Producer - " << producerStr_ << ", [numOfMsgPublished = " << numOfMsgPublished - << "] [numOfMsgAckSuccessfully = " << numOfMsgAckSuccessfully - << "] [numOfSendAsyncCalls =" << numOfSendAsyncCalls << "] [batchMessageContainer = " - << *batchMessageContainer << "]"); + LOG_INFO("Producer - " << producerStr_ << + ", [batchMessageContainer = " << *batchMessageContainer << "]"); } else { - LOG_INFO("Producer - " << producerStr_ << ", [numOfMsgPublished = " << numOfMsgPublished - << "] [numOfMsgAckSuccessfully = " << numOfMsgAckSuccessfully - << "] [numOfSendAsyncCalls =" << numOfSendAsyncCalls << "] [batching = off]"); + LOG_INFO("Producer - " << producerStr_ << + ", [batching = off]"); } } @@ -476,7 +483,6 @@ bool ProducerImpl::removeCorruptMessage(uint64_t sequenceId) { } else { LOG_DEBUG(getName() << "Remove corrupt message from queue " << sequenceId); pendingMessagesQueue_.pop(); - numOfMsgAckSuccessfully++; if (op.msg_.impl_->metadata.has_num_messages_in_batch()) { // batch message - need to release more spots // -1 since the pushing batch message into the queue already released a spot @@ -520,7 +526,6 @@ bool ProducerImpl::ackReceived(uint64_t sequenceId) { // Message was persisted correctly LOG_DEBUG(getName() << "Received ack for msg " << sequenceId); pendingMessagesQueue_.pop(); - numOfMsgAckSuccessfully++; if (op.msg_.impl_->metadata.has_num_messages_in_batch()) { // batch message - need to release more spots // -1 since the pushing batch message into the queue already released a spot diff --git a/pulsar-client-cpp/lib/ProducerImpl.h b/pulsar-client-cpp/lib/ProducerImpl.h index 89edae49b9f65..418d01dacd592 100644 --- a/pulsar-client-cpp/lib/ProducerImpl.h +++ b/pulsar-client-cpp/lib/ProducerImpl.h @@ -29,6 +29,8 @@ #include "HandlerBase.h" #include "SharedBuffer.h" #include "CompressionCodec.h" +#include "stats/ProducerStatsDisabled.h" +#include "stats/ProducerStatsImpl.h" using namespace pulsar; @@ -82,6 +84,7 @@ class ProducerImpl : public HandlerBase, public boost::enable_shared_from_this

MessageQueue; @@ -110,6 +113,8 @@ class ProducerImpl : public HandlerBase, public boost::enable_shared_from_this

producerCreatedPromise_; void failPendingMessages(Result result); - - unsigned long numOfMsgPublished; - unsigned long numOfSendAsyncCalls; - unsigned long numOfMsgAckSuccessfully; }; struct ProducerImplCmp { diff --git a/pulsar-client-cpp/lib/Result.cc b/pulsar-client-cpp/lib/Result.cc index 416d7102cda4f..e58beb9fb6ec5 100644 --- a/pulsar-client-cpp/lib/Result.cc +++ b/pulsar-client-cpp/lib/Result.cc @@ -25,7 +25,7 @@ using namespace pulsar; const char* pulsar::strResult(Result result) { switch (result) { case ResultOk: - return "OK"; + return "Ok"; case ResultUnknownError: return "UnknownError"; @@ -34,7 +34,7 @@ const char* pulsar::strResult(Result result) { return "InvalidConfiguration"; case ResultTimeout: - return "Timeout"; + return "TimeOut"; case ResultLookupError: return "LookupError"; diff --git a/pulsar-client-cpp/lib/Utils.h b/pulsar-client-cpp/lib/Utils.h index b0fe2ab3039db..b5fdaf3330897 100644 --- a/pulsar-client-cpp/lib/Utils.h +++ b/pulsar-client-cpp/lib/Utils.h @@ -67,6 +67,15 @@ struct WaitForCallbackType { } }; + +static std::ostream& operator<<(std::ostream& os, const std::map& m) { + os << "{"; + for (std::map::const_iterator it = m.begin(); it != m.end(); it++) { + os << "[Key: " << strResult(it->first) << ", Value: " << it->second << "], "; + } + os << "}"; + return os; +} } #endif /* UTILS_HPP_ */ diff --git a/pulsar-client-cpp/lib/stats/ConsumerStatsBase.h b/pulsar-client-cpp/lib/stats/ConsumerStatsBase.h new file mode 100644 index 0000000000000..b8b8fc959950a --- /dev/null +++ b/pulsar-client-cpp/lib/stats/ConsumerStatsBase.h @@ -0,0 +1,38 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +#ifndef PULSAR_CONSUMER_STATS_BASE_HEADER +#define PULSAR_CONSUMER_STATS_BASE_HEADER +#include +#include +#include +#include + +namespace pulsar { +class ConsumerStatsBase { + public: + virtual void receivedMessage(Message&, Result) = 0; + virtual void messageAcknowledged(Result, proto::CommandAck_AckType) = 0; + virtual ~ConsumerStatsBase() {} +}; + +typedef boost::shared_ptr ConsumerStatsBasePtr; +} + +#endif // PULSAR_CONSUMER_STATS_BASE_HEADER diff --git a/pulsar-client-cpp/lib/stats/ConsumerStatsDisabled.h b/pulsar-client-cpp/lib/stats/ConsumerStatsDisabled.h new file mode 100644 index 0000000000000..8f3e80c7b7dd5 --- /dev/null +++ b/pulsar-client-cpp/lib/stats/ConsumerStatsDisabled.h @@ -0,0 +1,35 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +#ifndef PULSAR_CONSUMER_STATS_DISABLED_H_ +#define PULSAR_CONSUMER_STATS_DISABLED_H_ + +#include + +namespace pulsar { + +class ConsumerStatsDisabled : public ConsumerStatsBase { + public: + virtual void receivedMessage(Message&, Result) {} + virtual void messageAcknowledged(Result, proto::CommandAck_AckType) {} +}; + +} /* namespace pulsar */ + +#endif /* PULSAR_CONSUMER_STATS_DISABLED_H_ */ diff --git a/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc b/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc new file mode 100644 index 0000000000000..1632d39fbf7ea --- /dev/null +++ b/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc @@ -0,0 +1,113 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 + +namespace pulsar { +DECLARE_LOG_OBJECT(); + +ConsumerStatsImpl::ConsumerStatsImpl(std::string consumerStr, DeadlineTimerPtr timer, + unsigned int statsIntervalInSeconds) + : consumerStr_(consumerStr), + timer_(timer), + statsIntervalInSeconds_(statsIntervalInSeconds), + totalNumBytesRecieved_(0), + numBytesRecieved_(0) { + timer_->expires_from_now(boost::posix_time::seconds(statsIntervalInSeconds_)); + timer_->async_wait( + boost::bind(&pulsar::ConsumerStatsImpl::flushAndReset, this, + boost::asio::placeholders::error)); +} + +ConsumerStatsImpl::ConsumerStatsImpl(const ConsumerStatsImpl& stats) + : consumerStr_(stats.consumerStr_), + totalNumBytesRecieved_(stats.totalNumBytesRecieved_), + numBytesRecieved_(stats.numBytesRecieved_), + receivedMsgMap_(stats.receivedMsgMap_), + ackedMsgMap_(stats.ackedMsgMap_), + totalReceivedMsgMap_(stats.totalReceivedMsgMap_), + totalAckedMsgMap_(stats.totalAckedMsgMap_), + statsIntervalInSeconds_(stats.statsIntervalInSeconds_) { +} + +void ConsumerStatsImpl::flushAndReset(const boost::system::error_code& ec) { + if (ec) { + LOG_DEBUG("Ignoring timer cancelled event, code[" << ec <<"]"); + return; + } + + Lock lock(mutex_); + ConsumerStatsImpl tmp = *this; + numBytesRecieved_ = 0; + receivedMsgMap_.clear(); + ackedMsgMap_.clear(); + lock.unlock(); + + timer_->expires_from_now(boost::posix_time::seconds(statsIntervalInSeconds_)); + timer_->async_wait( + boost::bind(&pulsar::ConsumerStatsImpl::flushAndReset, this, + boost::asio::placeholders::error)); + LOG_INFO(tmp); +} + +ConsumerStatsImpl::~ConsumerStatsImpl() { + Lock lock(mutex_); + if (timer_) { + timer_->cancel(); + } +} + +void ConsumerStatsImpl::receivedMessage(Message& msg, Result res) { + Lock lock(mutex_); + if (res == ResultOk) { + totalNumBytesRecieved_ += msg.getLength(); + numBytesRecieved_ += msg.getLength(); + } + receivedMsgMap_[res] += 1; + totalReceivedMsgMap_[res] += 1; +} + +void ConsumerStatsImpl::messageAcknowledged(Result res, proto::CommandAck_AckType ackType) { + Lock lock(mutex_); + ackedMsgMap_[std::make_pair(res, ackType)] += 1; + totalAckedMsgMap_[std::make_pair(res, ackType)] += 1; +} + +std::ostream& operator<<( + std::ostream& os, + const std::map, unsigned long>& m) { + os << "{"; + for (std::map, unsigned long>::const_iterator it = m.begin(); it != m.end(); it++) { + os << "[Key: {" << "Result: " << strResult((it->first).first) << ", ackType: " + << (it->first).second << "}, Value: " << it->second << "], "; + } + os << "}"; + return os; +} + +std::ostream& operator<<(std::ostream& os, const ConsumerStatsImpl& obj) { + os << "Consumer " << obj.consumerStr_ << ", ConsumerStatsImpl (" << "numBytesRecieved_ = " + << obj.numBytesRecieved_ << ", totalNumBytesRecieved_ = " << obj.totalNumBytesRecieved_ + << ", receivedMsgMap_ = " << obj.receivedMsgMap_ << ", ackedMsgMap_ = " << obj.ackedMsgMap_ + << ", totalReceivedMsgMap_ = " << obj.totalReceivedMsgMap_ << ", totalAckedMsgMap_ = " + << obj.totalAckedMsgMap_ << ")"; + return os; +} +} /* namespace pulsar */ diff --git a/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.h b/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.h new file mode 100644 index 0000000000000..98f27851818a2 --- /dev/null +++ b/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.h @@ -0,0 +1,83 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +#ifndef PULSAR_CONSUMER_STATS_IMPL_H_ +#define PULSAR_CONSUMER_STATS_IMPL_H_ + +#include +#include +#include +#include +#include +namespace pulsar { + +class ConsumerStatsImpl : public ConsumerStatsBase { + private: + unsigned long numBytesRecieved_; + std::map receivedMsgMap_; + std::map, unsigned long> ackedMsgMap_; + + unsigned long totalNumBytesRecieved_; + std::map totalReceivedMsgMap_; + std::map, unsigned long> totalAckedMsgMap_; + + std::string consumerStr_; + DeadlineTimerPtr timer_; + boost::mutex mutex_; + unsigned int statsIntervalInSeconds_; + + friend std::ostream& operator<<(std::ostream&, const ConsumerStatsImpl&); + friend std::ostream& operator<<(std::ostream&, const std::map&); + friend class PulsarFriend; + public: + ConsumerStatsImpl(std::string, DeadlineTimerPtr, unsigned int); + ConsumerStatsImpl(const ConsumerStatsImpl& stats); + void flushAndReset(const boost::system::error_code&); + virtual void receivedMessage(Message&, Result); + virtual void messageAcknowledged(Result, proto::CommandAck_AckType); + virtual ~ConsumerStatsImpl(); + + const inline std::map, unsigned long>& getAckedMsgMap() const { + return ackedMsgMap_; + } + + inline unsigned long getNumBytesRecieved() const { + return numBytesRecieved_; + } + + const inline std::map& getReceivedMsgMap() const { + return receivedMsgMap_; + } + + inline const std::map, unsigned long>& getTotalAckedMsgMap() const { + return totalAckedMsgMap_; + } + + inline unsigned long getTotalNumBytesRecieved() const { + return totalNumBytesRecieved_; + } + + const inline std::map& getTotalReceivedMsgMap() const { + return totalReceivedMsgMap_; + } +}; +typedef boost::shared_ptr ConsumerStatsImplPtr; +} /* namespace pulsar */ + +#endif /* PULSAR_CONSUMER_STATS_IMPL_H_ */ diff --git a/pulsar-client-cpp/lib/stats/ProducerStatsBase.h b/pulsar-client-cpp/lib/stats/ProducerStatsBase.h new file mode 100644 index 0000000000000..314fe31234cb1 --- /dev/null +++ b/pulsar-client-cpp/lib/stats/ProducerStatsBase.h @@ -0,0 +1,37 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +#ifndef PULSAR_PRODUCER_STATS_BASE_HEADER +#define PULSAR_PRODUCER_STATS_BASE_HEADER +#include +#include +#include + +namespace pulsar { +class ProducerStatsBase { + public: + virtual void messageSent(const Message& msg) = 0; + virtual void messageReceived(Result&, boost::posix_time::ptime&) = 0; + virtual ~ProducerStatsBase() {}; +}; + +typedef boost::shared_ptr ProducerStatsBasePtr; +} + +#endif // PULSAR_PRODUCER_STATS_BASE_HEADER diff --git a/pulsar-client-cpp/lib/stats/ProducerStatsDisabled.h b/pulsar-client-cpp/lib/stats/ProducerStatsDisabled.h new file mode 100644 index 0000000000000..0b4c3504b513e --- /dev/null +++ b/pulsar-client-cpp/lib/stats/ProducerStatsDisabled.h @@ -0,0 +1,32 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +#ifndef PULSAR_PRODUCER_STATS_DISABLED_HEADER +#define PULSAR_PRODUCER_STATS_DISABLED_HEADER +#include + +namespace pulsar { +class ProducerStatsDisabled : public ProducerStatsBase { + public: + virtual void messageSent(const Message& msg) {}; + virtual void messageReceived(Result&, boost::posix_time::ptime&) {}; +}; + +} +#endif // PULSAR_PRODUCER_STATS_DISABLED_HEADER diff --git a/pulsar-client-cpp/lib/stats/ProducerStatsImpl.cc b/pulsar-client-cpp/lib/stats/ProducerStatsImpl.cc new file mode 100644 index 0000000000000..fcdf5724ace79 --- /dev/null +++ b/pulsar-client-cpp/lib/stats/ProducerStatsImpl.cc @@ -0,0 +1,132 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 + +namespace pulsar { +DECLARE_LOG_OBJECT(); + +static const boost::array probs = { 0.5, 0.9, 0.99, 0.999 }; + +std::string ProducerStatsImpl::latencyToString(const LatencyAccumulator& obj) { + boost::accumulators::detail::extractor_result::type latencies = + boost::accumulators::extended_p_square(obj); + std::stringstream os; + os << "Latencies [ 50pct: " << latencies[0] / 1e3 << "ms" << ", 90pct: " << latencies[1] / 1e3 + << "ms" << ", 99pct: " << latencies[2] / 1e3 << "ms" << ", 99.9pct: " + << latencies[3] / 1e3 << "ms" << "]"; + return os.str(); +} + +ProducerStatsImpl::ProducerStatsImpl(std::string producerStr, DeadlineTimerPtr timer, + unsigned int statsIntervalInSeconds) + : numMsgsSent_(0), + numBytesSent_(0), + totalMsgsSent_(0), + totalBytesSent_(0), + timer_(timer), + producerStr_(producerStr), + statsIntervalInSeconds_(statsIntervalInSeconds), + mutex_(), + latencyAccumulator_(boost::accumulators::tag::extended_p_square::probabilities = probs), + totalLatencyAccumulator_(boost::accumulators::tag::extended_p_square::probabilities = + probs) { + timer_->expires_from_now(boost::posix_time::seconds(statsIntervalInSeconds_)); + timer_->async_wait( + boost::bind(&pulsar::ProducerStatsImpl::flushAndReset, this, + boost::asio::placeholders::error)); +} + +ProducerStatsImpl::ProducerStatsImpl(const ProducerStatsImpl& stats) + : numMsgsSent_(stats.numMsgsSent_), + numBytesSent_(stats.numBytesSent_), + totalMsgsSent_(stats.totalMsgsSent_), + totalBytesSent_(stats.totalBytesSent_), + sendMap_(stats.sendMap_), + totalSendMap_(stats.totalSendMap_), + timer_(), + producerStr_(stats.producerStr_), + statsIntervalInSeconds_(stats.statsIntervalInSeconds_), + mutex_(), + latencyAccumulator_(stats.latencyAccumulator_), + totalLatencyAccumulator_(stats.totalLatencyAccumulator_) { +} + +void ProducerStatsImpl::flushAndReset(const boost::system::error_code& ec) { + if (ec) { + LOG_DEBUG("Ignoring timer cancelled event, code[" << ec <<"]"); + return; + } + + Lock lock(mutex_); + ProducerStatsImpl tmp = *this; + numMsgsSent_ = 0; + numBytesSent_ = 0; + sendMap_.clear(); + latencyAccumulator_ = LatencyAccumulator( + boost::accumulators::tag::extended_p_square::probabilities = probs); + lock.unlock(); + + timer_->expires_from_now(boost::posix_time::seconds(statsIntervalInSeconds_)); + timer_->async_wait( + boost::bind(&pulsar::ProducerStatsImpl::flushAndReset, this, + boost::asio::placeholders::error)); + LOG_INFO(tmp); +} + +void ProducerStatsImpl::messageSent(const Message& msg) { + Lock lock(mutex_); + numMsgsSent_++; + totalMsgsSent_++; + numBytesSent_ += msg.getLength(); + totalBytesSent_ += msg.getLength(); +} + +void ProducerStatsImpl::messageReceived(Result& res, boost::posix_time::ptime& publishTime) { + boost::posix_time::ptime currentTime = boost::posix_time::microsec_clock::universal_time(); + double diffInMicros = (currentTime - publishTime).total_microseconds(); + Lock lock(mutex_); + totalLatencyAccumulator_(diffInMicros); + latencyAccumulator_(diffInMicros); + sendMap_[res] += 1; // Value will automatically be initialized to 0 in the constructor + totalSendMap_[res] += 1; // Value will automatically be initialized to 0 in the constructor +} + +ProducerStatsImpl::~ProducerStatsImpl() { + Lock lock(mutex_); + if (timer_) { + timer_->cancel(); + } +} + +std::ostream& operator<<(std::ostream& os, const ProducerStatsImpl& obj) { + os << "Producer " << obj.producerStr_ << ", ProducerStatsImpl (" << "numMsgsSent_ = " + << obj.numMsgsSent_ << ", numBytesSent_ = " << obj.numBytesSent_ << ", sendMap_ = " + << obj.sendMap_ << ", latencyAccumulator_ = " + << ProducerStatsImpl::latencyToString(obj.latencyAccumulator_) << ", totalMsgsSent_ = " + << obj.totalMsgsSent_ << ", totalBytesSent_ = " << obj.totalBytesSent_ + << ", totalAcksReceived_ = " << ", totalSendMap_ = " << obj.totalSendMap_ + << ", totalLatencyAccumulator_ = " + << ProducerStatsImpl::latencyToString(obj.totalLatencyAccumulator_) << ")"; + return os; +} +} diff --git a/pulsar-client-cpp/lib/stats/ProducerStatsImpl.h b/pulsar-client-cpp/lib/stats/ProducerStatsImpl.h new file mode 100644 index 0000000000000..f9b92be9a856d --- /dev/null +++ b/pulsar-client-cpp/lib/stats/ProducerStatsImpl.h @@ -0,0 +1,118 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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. + */ + +#ifndef PULSAR_PRODUCER_STATS_IMPL_HEADER +#define PULSAR_PRODUCER_STATS_IMPL_HEADER + +#include +#include +#include +#include + +#include +#include +#include +#include +#include + +#include +#include +#include +#include +#include +#include +#include + +namespace pulsar { +typedef boost::accumulators::accumulator_set > LatencyAccumulator; + +class ProducerStatsImpl : public boost::enable_shared_from_this, + public ProducerStatsBase { + private: + unsigned long numMsgsSent_; + unsigned long numBytesSent_; + std::map sendMap_; + LatencyAccumulator latencyAccumulator_; + + unsigned long totalMsgsSent_; + unsigned long totalBytesSent_; + std::map totalSendMap_; + LatencyAccumulator totalLatencyAccumulator_; + + std::string producerStr_; + DeadlineTimerPtr timer_; + boost::mutex mutex_; + unsigned int statsIntervalInSeconds_; + + friend std::ostream& operator<<(std::ostream&, const ProducerStatsImpl&); + friend std::ostream& operator<<(std::ostream&, const std::map&); + friend class PulsarFriend; + + static std::string latencyToString(const LatencyAccumulator&); + public: + ProducerStatsImpl(std::string, DeadlineTimerPtr, unsigned int); + + ProducerStatsImpl(const ProducerStatsImpl& stats); + + void flushAndReset(const boost::system::error_code&); + + void messageSent(const Message&); + + void messageReceived(Result&, boost::posix_time::ptime&); + + ~ProducerStatsImpl(); + + inline unsigned long getNumMsgsSent() { + return numMsgsSent_; + } + + inline unsigned long getNumBytesSent() { + return numBytesSent_; + } + + inline std::map getSendMap() { + return sendMap_; + } + + inline unsigned long getTotalMsgsSent() { + return totalMsgsSent_; + } + + inline unsigned long getTotalBytesSent() { + return totalBytesSent_; + } + + inline std::map getTotalSendMap() { + return totalSendMap_; + } + + inline LatencyAccumulator getLatencyAccumulator() { + return latencyAccumulator_; + } + + inline LatencyAccumulator getTotalLatencyAccumulator() { + return totalLatencyAccumulator_; + } +}; +typedef boost::shared_ptr ProducerStatsImplPtr; +} + +#endif // PULSAR_PRODUCER_STATS_IMPL_HEADER diff --git a/pulsar-client-cpp/log b/pulsar-client-cpp/log new file mode 100644 index 0000000000000..f7c32fdc736aa --- /dev/null +++ b/pulsar-client-cpp/log @@ -0,0 +1,1202 @@ +[==========] Running 72 tests from 16 test cases. +[----------] Global test environment set-up. +[----------] 3 tests from AuthPluginTest +[ RUN ] AuthPluginTest.testCreate +/Users/jai1/cpp/pulsar/pulsar-client-cpp/tests/AuthPluginTest.cc:44: Failure +Value of: auth != NULL + Actual: false +Expected: true +[ FAILED ] AuthPluginTest.testCreate (1 ms) +[ RUN ] AuthPluginTest.testTls +2017-06-20 18:03:08.593 : ERROR /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientConnection.cc(146)- ../../pulsar-broker/src/test/resources/authentication/tls/cacert.pem: No such trustCertFile +2017-06-20 18:03:08.593 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientConnection.cc(1127)- [ -> pulsar+ssl://localhost:9886] Connection closed +2017-06-20 18:03:08.593 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConnectionPool.cc(62)- Created connection for pulsar+ssl://localhost:9886 +2017-06-20 18:03:08.593 : ERROR /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientImpl.cc(144)- Error Checking/Getting Partition Metadata while creating producer on persistent://property/cluster/namespace/test-tls -- 5 +/Users/jai1/cpp/pulsar/pulsar-client-cpp/tests/AuthPluginTest.cc:72: Failure + Expected: ResultOk + Which is: Ok +To be equal to: result + Which is: ConnectError +2017-06-20 18:03:08.593 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientImpl.cc(84)- ~ClientImpl +[ FAILED ] AuthPluginTest.testTls (3 ms) +[ RUN ] AuthPluginTest.testDisable +[ OK ] AuthPluginTest.testDisable (0 ms) +[----------] 3 tests from AuthPluginTest (4 ms total) + +[----------] 2 tests from BackoffTest +[ RUN ] BackoffTest.basicTest +[ OK ] BackoffTest.basicTest (0 ms) +[ RUN ] BackoffTest.maxTest +[ OK ] BackoffTest.maxTest (0 ms) +[----------] 2 tests from BackoffTest (0 ms total) + +[----------] 25 tests from BasicEndToEndTest +[ RUN ] BasicEndToEndTest.testBatchMessages +2017-06-20 18:03:08.595 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConnectionPool.cc(62)- Created connection for pulsar://localhost:8885 +2017-06-20 18:03:08.596 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientConnection.cc(265)- [127.0.0.1:50033 -> 127.0.0.1:8885] Connected to broker +2017-06-20 18:03:08.597 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/BatchMessageContainer.cc(39)- { BatchContainer [size = 0] [batchSizeInBytes_ = 0] [maxAllowedMessageBatchSizeInBytes_ = 131072] [maxAllowedNumMessagesInBatch_ = 2] [topicName = persistent://property/cluster/namespace/test-batch-messages] [producerName_ = ] [batchSizeInBytes_ = 0] [numberOfBatchesSent = 0] [averageBatchSize = 0]} BatchMessageContainer constructed +2017-06-20 18:03:08.597 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/test-batch-messages, ] Getting connection from pool +2017-06-20 18:03:08.599 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ProducerImpl.cc(118)- [persistent://property/cluster/namespace/test-batch-messages, ] Created producer on broker [127.0.0.1:50033 -> 127.0.0.1:8885] +2017-06-20 18:03:08.600 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/test-batch-messages, subscription-name, 0] Getting connection from pool +2017-06-20 18:03:08.601 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(155)- [persistent://property/cluster/namespace/test-batch-messages, subscription-name, 0] Created consumer on broker [127.0.0.1:50033 -> 127.0.0.1:8885] +2017-06-20 18:03:08.601 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(208)- [persistent://property/cluster/namespace/test-batch-messages, subscription-name, 0] Unsubscribing +2017-06-20 18:03:08.603 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(240)- [persistent://property/cluster/namespace/test-batch-messages, subscription-name, 0] Unsubscribed successfully +2017-06-20 18:03:08.604 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/test-batch-messages, subscription-name, 1] Getting connection from pool +2017-06-20 18:03:08.620 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(155)- [persistent://property/cluster/namespace/test-batch-messages, subscription-name, 1] Created consumer on broker [127.0.0.1:50033 -> 127.0.0.1:8885] +2017-06-20 18:03:14.338 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(81)- [persistent://property/cluster/namespace/test-batch-messages, subscription-name, 0] ~ConsumerImpl +2017-06-20 18:03:14.338 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(81)- [persistent://property/cluster/namespace/test-batch-messages, subscription-name, 1] ~ConsumerImpl +2017-06-20 18:03:14.338 : WARN /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(84)- [persistent://property/cluster/namespace/test-batch-messages, subscription-name, 1] Destroyed consumer which was not properly closed +2017-06-20 18:03:14.339 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientImpl.cc(84)- ~ClientImpl +[ OK ] BasicEndToEndTest.testBatchMessages (5745 ms) +[ RUN ] BasicEndToEndTest.testProduceConsume +2017-06-20 18:03:14.340 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConnectionPool.cc(62)- Created connection for pulsar://localhost:8885 +2017-06-20 18:03:14.341 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientConnection.cc(265)- [127.0.0.1:50037 -> 127.0.0.1:8885] Connected to broker +2017-06-20 18:03:14.342 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://prop/unit/ns1/my-topic, ] Getting connection from pool +2017-06-20 18:03:14.348 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ProducerImpl.cc(118)- [persistent://prop/unit/ns1/my-topic, ] Created producer on broker [127.0.0.1:50037 -> 127.0.0.1:8885] +2017-06-20 18:03:14.348 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://prop/unit/ns1/my-topic, my-sub-name, 0] Getting connection from pool +2017-06-20 18:03:14.351 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(155)- [persistent://prop/unit/ns1/my-topic, my-sub-name, 0] Created consumer on broker [127.0.0.1:50037 -> 127.0.0.1:8885] +2017-06-20 18:03:14.362 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(208)- [persistent://prop/unit/ns1/my-topic, my-sub-name, 0] Unsubscribing +2017-06-20 18:03:14.366 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(240)- [persistent://prop/unit/ns1/my-topic, my-sub-name, 0] Unsubscribed successfully +2017-06-20 18:03:14.367 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ProducerImpl.cc(407)- [persistent://prop/unit/ns1/my-topic, standalone-0-349] Closed producer +2017-06-20 18:03:14.367 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(81)- [persistent://prop/unit/ns1/my-topic, my-sub-name, 0] ~ConsumerImpl +2017-06-20 18:03:14.367 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientImpl.cc(84)- ~ClientImpl +[ OK ] BasicEndToEndTest.testProduceConsume (28 ms) +[ RUN ] BasicEndToEndTest.testLookupThrottling +2017-06-20 18:03:14.368 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConnectionPool.cc(62)- Created connection for pulsar://localhost:8885 +2017-06-20 18:03:14.369 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientConnection.cc(265)- [127.0.0.1:50039 -> 127.0.0.1:8885] Connected to broker +2017-06-20 18:03:14.369 : ERROR /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientImpl.cc(144)- Error Checking/Getting Partition Metadata while creating producer on persistent://prop/unit/ns1/my-topic-1 -- 19 +2017-06-20 18:03:14.370 : ERROR /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientImpl.cc(210)- Error Checking/Getting Partition Metadata while Subscribing- 19 +2017-06-20 18:03:14.370 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientImpl.cc(84)- ~ClientImpl +[ OK ] BasicEndToEndTest.testLookupThrottling (3 ms) +[ RUN ] BasicEndToEndTest.testNonExistingTopic +2017-06-20 18:03:14.370 : ERROR /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/DestinationName.cc(76)- Destination Name Invalid, does not have enough parts - persistent://prop/unit/ns1 +2017-06-20 18:03:14.370 : ERROR /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/DestinationName.cc(59)- Destination Name is not valid, topic name is empty - persistent://prop/unit/ns1 +2017-06-20 18:03:14.370 : ERROR /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/DestinationName.cc(153)- Destination Name Initialization failed +2017-06-20 18:03:14.370 : ERROR /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/DestinationName.cc(76)- Destination Name Invalid, does not have enough parts - persistent://prop/unit/ns1 +2017-06-20 18:03:14.370 : ERROR /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/DestinationName.cc(59)- Destination Name is not valid, topic name is empty - persistent://prop/unit/ns1 +2017-06-20 18:03:14.370 : ERROR /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/DestinationName.cc(153)- Destination Name Initialization failed +2017-06-20 18:03:14.370 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientImpl.cc(84)- ~ClientImpl +[ OK ] BasicEndToEndTest.testNonExistingTopic (0 ms) +[ RUN ] BasicEndToEndTest.testNonPersistentTopic +2017-06-20 18:03:14.370 : ERROR /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/DestinationName.cc(159)- Destination Name Validation Failed +2017-06-20 18:03:14.370 : ERROR /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/DestinationName.cc(159)- Destination Name Validation Failed +2017-06-20 18:03:14.370 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientImpl.cc(84)- ~ClientImpl +[ OK ] BasicEndToEndTest.testNonPersistentTopic (0 ms) +[ RUN ] BasicEndToEndTest.testSingleClientMultipleSubscriptions +2017-06-20 18:03:14.371 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConnectionPool.cc(62)- Created connection for pulsar://localhost:8885 +2017-06-20 18:03:14.372 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientConnection.cc(265)- [127.0.0.1:50041 -> 127.0.0.1:8885] Connected to broker +2017-06-20 18:03:14.372 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://prop/unit/ns1/my-topic-1, ] Getting connection from pool +2017-06-20 18:03:14.374 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ProducerImpl.cc(118)- [persistent://prop/unit/ns1/my-topic-1, ] Created producer on broker [127.0.0.1:50041 -> 127.0.0.1:8885] +2017-06-20 18:03:14.374 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://prop/unit/ns1/my-topic-1, my-sub-name, 0] Getting connection from pool +2017-06-20 18:03:14.375 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(155)- [persistent://prop/unit/ns1/my-topic-1, my-sub-name, 0] Created consumer on broker [127.0.0.1:50041 -> 127.0.0.1:8885] +2017-06-20 18:03:14.376 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://prop/unit/ns1/my-topic-1, my-sub-name, 1] Getting connection from pool +2017-06-20 18:03:14.377 : WARN /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientConnection.cc(836)- [127.0.0.1:50041 -> 127.0.0.1:8885] Received error response from server: 13 -- req_id: 2 +2017-06-20 18:03:14.377 : ERROR /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(199)- [persistent://prop/unit/ns1/my-topic-1, my-sub-name, 1] Failed to create consumer: ConsumerBusy +2017-06-20 18:03:14.377 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(81)- [persistent://prop/unit/ns1/my-topic-1, my-sub-name, 1] ~ConsumerImpl +2017-06-20 18:03:14.377 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(81)- [persistent://prop/unit/ns1/my-topic-1, my-sub-name, 0] ~ConsumerImpl +2017-06-20 18:03:14.377 : WARN /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(84)- [persistent://prop/unit/ns1/my-topic-1, my-sub-name, 0] Destroyed consumer which was not properly closed +2017-06-20 18:03:14.377 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientImpl.cc(84)- ~ClientImpl +[ OK ] BasicEndToEndTest.testSingleClientMultipleSubscriptions (7 ms) +[ RUN ] BasicEndToEndTest.testMultipleClientsMultipleSubscriptions +2017-06-20 18:03:14.378 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConnectionPool.cc(62)- Created connection for pulsar://localhost:8885 +2017-06-20 18:03:14.379 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientConnection.cc(265)- [127.0.0.1:50043 -> 127.0.0.1:8885] Connected to broker +2017-06-20 18:03:14.380 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://prop/unit/ns1/my-topic-2, ] Getting connection from pool +2017-06-20 18:03:14.381 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ProducerImpl.cc(118)- [persistent://prop/unit/ns1/my-topic-2, ] Created producer on broker [127.0.0.1:50043 -> 127.0.0.1:8885] +2017-06-20 18:03:14.382 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://prop/unit/ns1/my-topic-2, my-sub-name, 0] Getting connection from pool +2017-06-20 18:03:14.382 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(155)- [persistent://prop/unit/ns1/my-topic-2, my-sub-name, 0] Created consumer on broker [127.0.0.1:50043 -> 127.0.0.1:8885] +2017-06-20 18:03:14.384 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConnectionPool.cc(62)- Created connection for pulsar://localhost:8885 +2017-06-20 18:03:14.386 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientConnection.cc(265)- [127.0.0.1:50045 -> 127.0.0.1:8885] Connected to broker +2017-06-20 18:03:14.387 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://prop/unit/ns1/my-topic-2, my-sub-name, 0] Getting connection from pool +2017-06-20 18:03:14.390 : WARN /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientConnection.cc(836)- [127.0.0.1:50045 -> 127.0.0.1:8885] Received error response from server: 13 -- req_id: 0 +2017-06-20 18:03:14.390 : ERROR /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(199)- [persistent://prop/unit/ns1/my-topic-2, my-sub-name, 0] Failed to create consumer: ConsumerBusy +2017-06-20 18:03:14.390 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(81)- [persistent://prop/unit/ns1/my-topic-2, my-sub-name, 0] ~ConsumerImpl +2017-06-20 18:03:14.391 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ProducerImpl.cc(407)- [persistent://prop/unit/ns1/my-topic-2, standalone-0-351] Closed producer +2017-06-20 18:03:14.391 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(639)- [persistent://prop/unit/ns1/my-topic-2, my-sub-name, 0] Closed consumer 0 +2017-06-20 18:03:16.394 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(81)- [persistent://prop/unit/ns1/my-topic-2, my-sub-name, 0] ~ConsumerImpl +2017-06-20 18:03:16.394 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientImpl.cc(84)- ~ClientImpl +2017-06-20 18:03:16.394 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientImpl.cc(84)- ~ClientImpl +[ OK ] BasicEndToEndTest.testMultipleClientsMultipleSubscriptions (2017 ms) +[ RUN ] BasicEndToEndTest.testProduceAndConsumeAfterClientClose +2017-06-20 18:03:16.396 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConnectionPool.cc(62)- Created connection for pulsar://localhost:8885 +2017-06-20 18:03:16.397 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientConnection.cc(265)- [127.0.0.1:50048 -> 127.0.0.1:8885] Connected to broker +2017-06-20 18:03:16.398 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://prop/unit/ns1/my-topic-3, ] Getting connection from pool +2017-06-20 18:03:16.399 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ProducerImpl.cc(118)- [persistent://prop/unit/ns1/my-topic-3, ] Created producer on broker [127.0.0.1:50048 -> 127.0.0.1:8885] +2017-06-20 18:03:16.400 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://prop/unit/ns1/my-topic-3, my-sub-name, 0] Getting connection from pool +2017-06-20 18:03:16.401 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(155)- [persistent://prop/unit/ns1/my-topic-3, my-sub-name, 0] Created consumer on broker [127.0.0.1:50048 -> 127.0.0.1:8885] +2017-06-20 18:03:16.401 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(208)- [persistent://prop/unit/ns1/my-topic-3, my-sub-name, 0] Unsubscribing +2017-06-20 18:03:16.404 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(240)- [persistent://prop/unit/ns1/my-topic-3, my-sub-name, 0] Unsubscribed successfully +2017-06-20 18:03:16.404 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://prop/unit/ns1/my-topic-3, my-sub-name, 1] Getting connection from pool +2017-06-20 18:03:16.408 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(155)- [persistent://prop/unit/ns1/my-topic-3, my-sub-name, 1] Created consumer on broker [127.0.0.1:50048 -> 127.0.0.1:8885] +2017-06-20 18:03:16.408 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(81)- [persistent://prop/unit/ns1/my-topic-3, my-sub-name, 0] ~ConsumerImpl +2017-06-20 18:03:16.408 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/tests/BasicEndToEndTest.cc(289)- Publishing 10 messages synchronously +2017-06-20 18:03:16.425 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/tests/BasicEndToEndTest.cc(297)- Trying to receive 10 messages +2017-06-20 18:03:16.429 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/tests/BasicEndToEndTest.cc(307)- Closing client +2017-06-20 18:03:16.430 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ProducerImpl.cc(407)- [persistent://prop/unit/ns1/my-topic-3, standalone-0-352] Closed producer +2017-06-20 18:03:16.431 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(639)- [persistent://prop/unit/ns1/my-topic-3, my-sub-name, 1] Closed consumer 1 +2017-06-20 18:03:16.431 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/tests/BasicEndToEndTest.cc(310)- Trying to publish a message after closing the client +2017-06-20 18:03:16.431 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/tests/BasicEndToEndTest.cc(316)- Trying to consume a message after closing the client +2017-06-20 18:03:16.431 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(81)- [persistent://prop/unit/ns1/my-topic-3, my-sub-name, 1] ~ConsumerImpl +2017-06-20 18:03:16.431 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientImpl.cc(84)- ~ClientImpl +[ OK ] BasicEndToEndTest.testProduceAndConsumeAfterClientClose (37 ms) +[ RUN ] BasicEndToEndTest.testIamSoFancyCharactersInTopicName +2017-06-20 18:03:16.432 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConnectionPool.cc(62)- Created connection for pulsar://localhost:8885 +2017-06-20 18:03:16.433 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientConnection.cc(265)- [127.0.0.1:50050 -> 127.0.0.1:8885] Connected to broker +2017-06-20 18:03:16.434 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://prop/unit/ns1/topic@%*)(&!%$#@#$> 127.0.0.1:8885] +2017-06-20 18:03:16.436 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://prop/unit/ns1/topic@%*)(&!%$#@#$> 127.0.0.1:8885] +2017-06-20 18:03:16.437 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(81)- [persistent://prop/unit/ns1/topic@%*)(&!%$#@#$> 127.0.0.1:8885] Connected to broker +2017-06-20 18:03:16.440 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://prop/unit/ns1/::,::bf11, ] Getting connection from pool +2017-06-20 18:03:16.444 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ProducerImpl.cc(118)- [persistent://prop/unit/ns1/::,::bf11, ] Created producer on broker [127.0.0.1:50052 -> 127.0.0.1:8885] +2017-06-20 18:03:16.445 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://prop/unit/ns1/::,::bf11, weird-ass-characters-@%*)(&!%$#@#$> 127.0.0.1:8885] +2017-06-20 18:03:16.448 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(639)- [persistent://prop/unit/ns1/::,::bf11, weird-ass-characters-@%*)(&!%$#@#$> 127.0.0.1:8885] +2017-06-20 18:03:16.449 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(208)- [persistent://prop/unit/ns1/::,::bf11, weird-ass-characters-@%*)(&!%$#@#$> -> localhost:4080] Invalid Url protocol ''. Valid values are 'pulsar' and 'pulsar+ssl' +2017-06-20 18:03:16.453 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientConnection.cc(1127)- [ -> localhost:4080] Connection closed +2017-06-20 18:03:16.453 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientConnection.cc(179)- [ -> localhost:4080] Destroyed connection +2017-06-20 18:03:16.453 : ERROR /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientImpl.cc(144)- Error Checking/Getting Partition Metadata while creating producer on persistent://prop/unit/ns1/test -- 5 +2017-06-20 18:03:16.454 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConnectionPool.cc(62)- Created connection for test://localhost +2017-06-20 18:03:16.454 : ERROR /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientConnection.cc(344)- [ -> test://localhost] Invalid Url, unable to parse: system:0 Undefined error: 0 +2017-06-20 18:03:16.454 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientConnection.cc(1127)- [ -> test://localhost] Connection closed +2017-06-20 18:03:16.454 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientConnection.cc(179)- [ -> test://localhost] Destroyed connection +2017-06-20 18:03:16.454 : ERROR /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientImpl.cc(144)- Error Checking/Getting Partition Metadata while creating producer on persistent://prop/unit/ns1/test -- 5 +2017-06-20 18:03:16.455 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConnectionPool.cc(62)- Created connection for test://:4080 +2017-06-20 18:03:16.455 : ERROR /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientConnection.cc(344)- [ -> test://:4080] Invalid Url, unable to parse: system:0 Undefined error: 0 +2017-06-20 18:03:16.455 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientConnection.cc(1127)- [ -> test://:4080] Connection closed +2017-06-20 18:03:16.455 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientConnection.cc(179)- [ -> test://:4080] Destroyed connection +2017-06-20 18:03:16.456 : ERROR /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientImpl.cc(144)- Error Checking/Getting Partition Metadata while creating producer on persistent://prop/unit/ns1/test -- 5 +2017-06-20 18:03:16.457 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConnectionPool.cc(62)- Created connection for +2017-06-20 18:03:16.457 : ERROR /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientConnection.cc(344)- [ -> ] Invalid Url, unable to parse: system:0 Undefined error: 0 +2017-06-20 18:03:16.457 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientConnection.cc(1127)- [ -> ] Connection closed +2017-06-20 18:03:16.457 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientConnection.cc(179)- [ -> ] Destroyed connection +2017-06-20 18:03:16.457 : ERROR /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientImpl.cc(144)- Error Checking/Getting Partition Metadata while creating producer on persistent://prop/unit/ns1/test -- 5 +2017-06-20 18:03:16.458 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConnectionPool.cc(62)- Created connection for Dream of the day when this will be a valid URL +2017-06-20 18:03:16.458 : ERROR /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientConnection.cc(344)- [ -> Dream of the day when this will be a valid URL] Invalid Url, unable to parse: system:0 Undefined error: 0 +2017-06-20 18:03:16.458 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientConnection.cc(1127)- [ -> Dream of the day when this will be a valid URL] Connection closed +2017-06-20 18:03:16.458 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientConnection.cc(179)- [ -> Dream of the day when this will be a valid URL] Destroyed connection +2017-06-20 18:03:16.459 : ERROR /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientImpl.cc(144)- Error Checking/Getting Partition Metadata while creating producer on persistent://prop/unit/ns1/test -- 5 +2017-06-20 18:03:16.459 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientImpl.cc(84)- ~ClientImpl +2017-06-20 18:03:16.459 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientImpl.cc(84)- ~ClientImpl +2017-06-20 18:03:16.459 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientImpl.cc(84)- ~ClientImpl +2017-06-20 18:03:16.459 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientImpl.cc(84)- ~ClientImpl +2017-06-20 18:03:16.459 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientImpl.cc(84)- ~ClientImpl +[ OK ] BasicEndToEndTest.testInvalidUrlPassed (7 ms) +[ RUN ] BasicEndToEndTest.testPartitionedProducerConsumer +{"reason":"Partitioned topic already exist"}2017-06-20 18:03:16.462 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/tests/BasicEndToEndTest.cc(390)- res = 409 +2017-06-20 18:03:16.463 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConnectionPool.cc(62)- Created connection for pulsar://localhost:8885 +2017-06-20 18:03:16.464 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientConnection.cc(265)- [127.0.0.1:50056 -> 127.0.0.1:8885] Connected to broker +2017-06-20 18:03:16.465 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://prop/unit/ns/partition-test-partition-0, ] Getting connection from pool +2017-06-20 18:03:16.465 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://prop/unit/ns/partition-test-partition-1, ] Getting connection from pool +2017-06-20 18:03:16.465 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://prop/unit/ns/partition-test-partition-2, ] Getting connection from pool +2017-06-20 18:03:16.466 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ProducerImpl.cc(118)- [persistent://prop/unit/ns/partition-test-partition-0, ] Created producer on broker [127.0.0.1:50056 -> 127.0.0.1:8885] +2017-06-20 18:03:16.467 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ProducerImpl.cc(118)- [persistent://prop/unit/ns/partition-test-partition-1, ] Created producer on broker [127.0.0.1:50056 -> 127.0.0.1:8885] +2017-06-20 18:03:16.468 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ProducerImpl.cc(118)- [persistent://prop/unit/ns/partition-test-partition-2, ] Created producer on broker [127.0.0.1:50056 -> 127.0.0.1:8885] +2017-06-20 18:03:16.468 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://prop/unit/ns/partition-test-partition-0, subscription-A0, 0] Getting connection from pool +2017-06-20 18:03:16.468 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://prop/unit/ns/partition-test-partition-1, subscription-A1, 1] Getting connection from pool +2017-06-20 18:03:16.468 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://prop/unit/ns/partition-test-partition-2, subscription-A2, 2] Getting connection from pool +2017-06-20 18:03:16.469 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(155)- [persistent://prop/unit/ns/partition-test-partition-0, subscription-A0, 0] Created consumer on broker [127.0.0.1:50056 -> 127.0.0.1:8885] +2017-06-20 18:03:16.469 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(155)- [persistent://prop/unit/ns/partition-test-partition-1, subscription-A1, 1] Created consumer on broker [127.0.0.1:50056 -> 127.0.0.1:8885] +2017-06-20 18:03:16.470 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(155)- [persistent://prop/unit/ns/partition-test-partition-2, subscription-A2, 2] Created consumer on broker [127.0.0.1:50056 -> 127.0.0.1:8885] +2017-06-20 18:03:16.470 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/PartitionedConsumerImpl.cc(222)- Successfully Subscribed to Partitioned Topic - persistent://prop/unit/ns/partition-test with - 3 Partitions. +[ OK ] BasicEndToEndTest.testPartitionedProducerConsumer (36 ms) +[ RUN ] BasicEndToEndTest.testMessageTooBig +2017-06-20 18:03:16.496 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConnectionPool.cc(62)- Created connection for pulsar://localhost:8885 +2017-06-20 18:03:16.497 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientConnection.cc(265)- [127.0.0.1:50058 -> 127.0.0.1:8885] Connected to broker +2017-06-20 18:03:16.498 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://prop/unit/ns1/my-topic, ] Getting connection from pool +2017-06-20 18:03:16.500 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ProducerImpl.cc(118)- [persistent://prop/unit/ns1/my-topic, ] Created producer on broker [127.0.0.1:50058 -> 127.0.0.1:8885] +2017-06-20 18:03:16.541 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientImpl.cc(84)- ~ClientImpl +[ OK ] BasicEndToEndTest.testMessageTooBig (46 ms) +[ RUN ] BasicEndToEndTest.testCompressionLZ4 +2017-06-20 18:03:16.542 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConnectionPool.cc(62)- Created connection for pulsar://localhost:8885 +2017-06-20 18:03:16.543 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientConnection.cc(265)- [127.0.0.1:50060 -> 127.0.0.1:8885] Connected to broker +2017-06-20 18:03:16.544 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://prop/unit/namespace1/my-topic-lz4, ] Getting connection from pool +2017-06-20 18:03:16.548 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ProducerImpl.cc(118)- [persistent://prop/unit/namespace1/my-topic-lz4, ] Created producer on broker [127.0.0.1:50060 -> 127.0.0.1:8885] +2017-06-20 18:03:16.548 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://prop/unit/namespace1/my-topic-lz4, my-sub-name, 0] Getting connection from pool +2017-06-20 18:03:16.552 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(155)- [persistent://prop/unit/namespace1/my-topic-lz4, my-sub-name, 0] Created consumer on broker [127.0.0.1:50060 -> 127.0.0.1:8885] +2017-06-20 18:03:16.563 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(208)- [persistent://prop/unit/namespace1/my-topic-lz4, my-sub-name, 0] Unsubscribing +2017-06-20 18:03:16.567 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(240)- [persistent://prop/unit/namespace1/my-topic-lz4, my-sub-name, 0] Unsubscribed successfully +2017-06-20 18:03:16.567 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ProducerImpl.cc(407)- [persistent://prop/unit/namespace1/my-topic-lz4, standalone-0-359] Closed producer +2017-06-20 18:03:16.567 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(81)- [persistent://prop/unit/namespace1/my-topic-lz4, my-sub-name, 0] ~ConsumerImpl +2017-06-20 18:03:16.567 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientImpl.cc(84)- ~ClientImpl +[ OK ] BasicEndToEndTest.testCompressionLZ4 (27 ms) +[ RUN ] BasicEndToEndTest.testCompressionZLib +2017-06-20 18:03:16.569 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConnectionPool.cc(62)- Created connection for pulsar://localhost:8885 +2017-06-20 18:03:16.570 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientConnection.cc(265)- [127.0.0.1:50062 -> 127.0.0.1:8885] Connected to broker +2017-06-20 18:03:16.571 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://prop/unit/ns1/my-topic-zlib, ] Getting connection from pool +2017-06-20 18:03:16.575 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ProducerImpl.cc(118)- [persistent://prop/unit/ns1/my-topic-zlib, ] Created producer on broker [127.0.0.1:50062 -> 127.0.0.1:8885] +2017-06-20 18:03:16.575 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://prop/unit/ns1/my-topic-zlib, my-sub-name, 0] Getting connection from pool +2017-06-20 18:03:16.578 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(155)- [persistent://prop/unit/ns1/my-topic-zlib, my-sub-name, 0] Created consumer on broker [127.0.0.1:50062 -> 127.0.0.1:8885] +2017-06-20 18:03:16.589 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(208)- [persistent://prop/unit/ns1/my-topic-zlib, my-sub-name, 0] Unsubscribing +2017-06-20 18:03:16.593 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(240)- [persistent://prop/unit/ns1/my-topic-zlib, my-sub-name, 0] Unsubscribed successfully +2017-06-20 18:03:16.593 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ProducerImpl.cc(407)- [persistent://prop/unit/ns1/my-topic-zlib, standalone-0-360] Closed producer +2017-06-20 18:03:16.593 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(81)- [persistent://prop/unit/ns1/my-topic-zlib, my-sub-name, 0] ~ConsumerImpl +2017-06-20 18:03:16.593 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientImpl.cc(84)- ~ClientImpl +[ OK ] BasicEndToEndTest.testCompressionZLib (26 ms) +[ RUN ] BasicEndToEndTest.testConfigurationFile +[ OK ] BasicEndToEndTest.testConfigurationFile (0 ms) +[ RUN ] BasicEndToEndTest.testSinglePartitionRoutingPolicy +{"reason":"Partitioned topic already exist"}2017-06-20 18:03:16.597 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/tests/BasicEndToEndTest.cc(550)- res = 409 +2017-06-20 18:03:16.598 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConnectionPool.cc(62)- Created connection for pulsar://localhost:8885 +2017-06-20 18:03:16.599 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientConnection.cc(265)- [127.0.0.1:50066 -> 127.0.0.1:8885] Connected to broker +2017-06-20 18:03:16.600 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://prop/unit/ns/partition-testSinglePartitionRoutingPolicy-partition-0, ] Getting connection from pool +2017-06-20 18:03:16.600 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://prop/unit/ns/partition-testSinglePartitionRoutingPolicy-partition-1, ] Getting connection from pool +2017-06-20 18:03:16.600 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://prop/unit/ns/partition-testSinglePartitionRoutingPolicy-partition-2, ] Getting connection from pool +2017-06-20 18:03:16.600 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://prop/unit/ns/partition-testSinglePartitionRoutingPolicy-partition-3, ] Getting connection from pool +2017-06-20 18:03:16.600 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://prop/unit/ns/partition-testSinglePartitionRoutingPolicy-partition-4, ] Getting connection from pool +2017-06-20 18:03:16.602 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ProducerImpl.cc(118)- [persistent://prop/unit/ns/partition-testSinglePartitionRoutingPolicy-partition-0, ] Created producer on broker [127.0.0.1:50066 -> 127.0.0.1:8885] +2017-06-20 18:03:16.603 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ProducerImpl.cc(118)- [persistent://prop/unit/ns/partition-testSinglePartitionRoutingPolicy-partition-1, ] Created producer on broker [127.0.0.1:50066 -> 127.0.0.1:8885] +2017-06-20 18:03:16.603 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ProducerImpl.cc(118)- [persistent://prop/unit/ns/partition-testSinglePartitionRoutingPolicy-partition-2, ] Created producer on broker [127.0.0.1:50066 -> 127.0.0.1:8885] +2017-06-20 18:03:16.604 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ProducerImpl.cc(118)- [persistent://prop/unit/ns/partition-testSinglePartitionRoutingPolicy-partition-3, ] Created producer on broker [127.0.0.1:50066 -> 127.0.0.1:8885] +2017-06-20 18:03:16.604 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ProducerImpl.cc(118)- [persistent://prop/unit/ns/partition-testSinglePartitionRoutingPolicy-partition-4, ] Created producer on broker [127.0.0.1:50066 -> 127.0.0.1:8885] +2017-06-20 18:03:16.605 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://prop/unit/ns/partition-testSinglePartitionRoutingPolicy-partition-0, subscription-A0, 0] Getting connection from pool +2017-06-20 18:03:16.605 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://prop/unit/ns/partition-testSinglePartitionRoutingPolicy-partition-1, subscription-A1, 1] Getting connection from pool +2017-06-20 18:03:16.605 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://prop/unit/ns/partition-testSinglePartitionRoutingPolicy-partition-2, subscription-A2, 2] Getting connection from pool +2017-06-20 18:03:16.605 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://prop/unit/ns/partition-testSinglePartitionRoutingPolicy-partition-3, subscription-A3, 3] Getting connection from pool +2017-06-20 18:03:16.605 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://prop/unit/ns/partition-testSinglePartitionRoutingPolicy-partition-4, subscription-A4, 4] Getting connection from pool +2017-06-20 18:03:16.606 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(155)- [persistent://prop/unit/ns/partition-testSinglePartitionRoutingPolicy-partition-0, subscription-A0, 0] Created consumer on broker [127.0.0.1:50066 -> 127.0.0.1:8885] +2017-06-20 18:03:16.606 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(155)- [persistent://prop/unit/ns/partition-testSinglePartitionRoutingPolicy-partition-1, subscription-A1, 1] Created consumer on broker [127.0.0.1:50066 -> 127.0.0.1:8885] +2017-06-20 18:03:16.606 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(155)- [persistent://prop/unit/ns/partition-testSinglePartitionRoutingPolicy-partition-2, subscription-A2, 2] Created consumer on broker [127.0.0.1:50066 -> 127.0.0.1:8885] +2017-06-20 18:03:16.607 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(155)- [persistent://prop/unit/ns/partition-testSinglePartitionRoutingPolicy-partition-3, subscription-A3, 3] Created consumer on broker [127.0.0.1:50066 -> 127.0.0.1:8885] +2017-06-20 18:03:16.607 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(155)- [persistent://prop/unit/ns/partition-testSinglePartitionRoutingPolicy-partition-4, subscription-A4, 4] Created consumer on broker [127.0.0.1:50066 -> 127.0.0.1:8885] +2017-06-20 18:03:16.607 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/PartitionedConsumerImpl.cc(222)- Successfully Subscribed to Partitioned Topic - persistent://prop/unit/ns/partition-testSinglePartitionRoutingPolicy with - 5 Partitions. +2017-06-20 18:03:16.626 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(639)- [persistent://prop/unit/ns/partition-testSinglePartitionRoutingPolicy-partition-0, subscription-A0, 0] Closed consumer 0 +2017-06-20 18:03:16.626 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(639)- [persistent://prop/unit/ns/partition-testSinglePartitionRoutingPolicy-partition-1, subscription-A1, 1] Closed consumer 1 +2017-06-20 18:03:16.626 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(639)- [persistent://prop/unit/ns/partition-testSinglePartitionRoutingPolicy-partition-2, subscription-A2, 2] Closed consumer 2 +2017-06-20 18:03:16.626 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(639)- [persistent://prop/unit/ns/partition-testSinglePartitionRoutingPolicy-partition-3, subscription-A3, 3] Closed consumer 3 +2017-06-20 18:03:16.627 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(639)- [persistent://prop/unit/ns/partition-testSinglePartitionRoutingPolicy-partition-4, subscription-A4, 4] Closed consumer 4 +2017-06-20 18:03:16.627 : ERROR /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/PartitionedProducerImpl.cc(178)- Closing the producer failed for partition - 0 +2017-06-20 18:03:16.627 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ProducerImpl.cc(407)- [persistent://prop/unit/ns/partition-testSinglePartitionRoutingPolicy-partition-0, standalone-0-361] Closed producer +[ OK ] BasicEndToEndTest.testSinglePartitionRoutingPolicy (33 ms) +[ RUN ] BasicEndToEndTest.testNamespaceName +[ OK ] BasicEndToEndTest.testNamespaceName (0 ms) +[ RUN ] BasicEndToEndTest.testConsumerClose +2017-06-20 18:03:16.627 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ProducerImpl.cc(407)- [persistent://prop/unit/ns/partition-testSinglePartitionRoutingPolicy-partition-1, standalone-0-362] Closed producer +2017-06-20 18:03:16.627 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ProducerImpl.cc(407)- [persistent://prop/unit/ns/partition-testSinglePartitionRoutingPolicy-partition-2, standalone-0-363] Closed producer +2017-06-20 18:03:16.628 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ProducerImpl.cc(407)- [persistent://prop/unit/ns/partition-testSinglePartitionRoutingPolicy-partition-3, standalone-0-364] Closed producer +2017-06-20 18:03:16.628 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ProducerImpl.cc(407)- [persistent://prop/unit/ns/partition-testSinglePartitionRoutingPolicy-partition-4, standalone-0-365] Closed producer +2017-06-20 18:03:16.628 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConnectionPool.cc(62)- Created connection for pulsar://localhost:8885 +2017-06-20 18:03:16.629 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientConnection.cc(265)- [127.0.0.1:50068 -> 127.0.0.1:8885] Connected to broker +2017-06-20 18:03:16.630 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://prop/unit/ns1/testConsumerClose, my-sub-name, 0] Getting connection from pool +2017-06-20 18:03:16.631 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(155)- [persistent://prop/unit/ns1/testConsumerClose, my-sub-name, 0] Created consumer on broker [127.0.0.1:50068 -> 127.0.0.1:8885] +2017-06-20 18:03:16.631 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(639)- [persistent://prop/unit/ns1/testConsumerClose, my-sub-name, 0] Closed consumer 0 +2017-06-20 18:03:16.631 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(81)- [persistent://prop/unit/ns1/testConsumerClose, my-sub-name, 0] ~ConsumerImpl +2017-06-20 18:03:16.631 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientImpl.cc(84)- ~ClientImpl +[ OK ] BasicEndToEndTest.testConsumerClose (4 ms) +[ RUN ] BasicEndToEndTest.testDuplicateConsumerCreationOnPartitionedTopic +{"reason":"Partitioned topic already exist"}2017-06-20 18:03:16.635 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/tests/BasicEndToEndTest.cc(611)- res = 409 +2017-06-20 18:03:18.642 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConnectionPool.cc(62)- Created connection for pulsar://localhost:8885 +2017-06-20 18:03:18.643 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientConnection.cc(265)- [127.0.0.1:50072 -> 127.0.0.1:8885] Connected to broker +2017-06-20 18:03:18.644 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://prop/unit/ns/partition-testDuplicateConsumerCreationOnPartitionedTopic, ] Getting connection from pool +2017-06-20 18:03:18.645 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ProducerImpl.cc(118)- [persistent://prop/unit/ns/partition-testDuplicateConsumerCreationOnPartitionedTopic, ] Created producer on broker [127.0.0.1:50072 -> 127.0.0.1:8885] +2017-06-20 18:03:18.664 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/tests/BasicEndToEndTest.cc(633)- Creating Subscriber +2017-06-20 18:03:18.665 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://prop/unit/ns/partition-testDuplicateConsumerCreationOnPartitionedTopic, CONSUMER, 0] Getting connection from pool +2017-06-20 18:03:18.666 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(155)- [persistent://prop/unit/ns/partition-testDuplicateConsumerCreationOnPartitionedTopic, CONSUMER, 0] Created consumer on broker [127.0.0.1:50072 -> 127.0.0.1:8885] +2017-06-20 18:03:18.666 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/tests/BasicEndToEndTest.cc(644)- Creating Another Subscriber +2017-06-20 18:03:18.667 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://prop/unit/ns/partition-testDuplicateConsumerCreationOnPartitionedTopic, CONSUMER, 1] Getting connection from pool +2017-06-20 18:03:18.669 : WARN /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientConnection.cc(836)- [127.0.0.1:50072 -> 127.0.0.1:8885] Received error response from server: 13 -- req_id: 2 +2017-06-20 18:03:18.669 : ERROR /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(199)- [persistent://prop/unit/ns/partition-testDuplicateConsumerCreationOnPartitionedTopic, CONSUMER, 1] Failed to create consumer: ConsumerBusy +2017-06-20 18:03:18.669 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(81)- [persistent://prop/unit/ns/partition-testDuplicateConsumerCreationOnPartitionedTopic, CONSUMER, 1] ~ConsumerImpl +2017-06-20 18:03:18.669 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(639)- [persistent://prop/unit/ns/partition-testDuplicateConsumerCreationOnPartitionedTopic, CONSUMER, 0] Closed consumer 0 +2017-06-20 18:03:18.670 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ProducerImpl.cc(407)- [persistent://prop/unit/ns/partition-testDuplicateConsumerCreationOnPartitionedTopic, standalone-0-366] Closed producer +2017-06-20 18:03:18.670 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(81)- [persistent://prop/unit/ns/partition-testDuplicateConsumerCreationOnPartitionedTopic, CONSUMER, 0] ~ConsumerImpl +2017-06-20 18:03:18.670 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientImpl.cc(84)- ~ClientImpl +[ OK ] BasicEndToEndTest.testDuplicateConsumerCreationOnPartitionedTopic (2039 ms) +[ RUN ] BasicEndToEndTest.testRoundRobinRoutingPolicy +{"reason":"Partitioned topic already exist"}2017-06-20 18:03:18.674 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/tests/BasicEndToEndTest.cc(662)- res = 409 +2017-06-20 18:03:18.675 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConnectionPool.cc(62)- Created connection for pulsar://localhost:8885 +2017-06-20 18:03:18.676 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientConnection.cc(265)- [127.0.0.1:50076 -> 127.0.0.1:8885] Connected to broker +2017-06-20 18:03:18.676 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://prop/unit/ns/partition-testRoundRobinRoutingPolicy-partition-0, ] Getting connection from pool +2017-06-20 18:03:18.677 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://prop/unit/ns/partition-testRoundRobinRoutingPolicy-partition-1, ] Getting connection from pool +2017-06-20 18:03:18.677 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://prop/unit/ns/partition-testRoundRobinRoutingPolicy-partition-2, ] Getting connection from pool +2017-06-20 18:03:18.677 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://prop/unit/ns/partition-testRoundRobinRoutingPolicy-partition-3, ] Getting connection from pool +2017-06-20 18:03:18.677 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://prop/unit/ns/partition-testRoundRobinRoutingPolicy-partition-4, ] Getting connection from pool +2017-06-20 18:03:18.678 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ProducerImpl.cc(118)- [persistent://prop/unit/ns/partition-testRoundRobinRoutingPolicy-partition-0, ] Created producer on broker [127.0.0.1:50076 -> 127.0.0.1:8885] +2017-06-20 18:03:18.679 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ProducerImpl.cc(118)- [persistent://prop/unit/ns/partition-testRoundRobinRoutingPolicy-partition-1, ] Created producer on broker [127.0.0.1:50076 -> 127.0.0.1:8885] +2017-06-20 18:03:18.680 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ProducerImpl.cc(118)- [persistent://prop/unit/ns/partition-testRoundRobinRoutingPolicy-partition-2, ] Created producer on broker [127.0.0.1:50076 -> 127.0.0.1:8885] +2017-06-20 18:03:18.680 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ProducerImpl.cc(118)- [persistent://prop/unit/ns/partition-testRoundRobinRoutingPolicy-partition-3, ] Created producer on broker [127.0.0.1:50076 -> 127.0.0.1:8885] +2017-06-20 18:03:18.681 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ProducerImpl.cc(118)- [persistent://prop/unit/ns/partition-testRoundRobinRoutingPolicy-partition-4, ] Created producer on broker [127.0.0.1:50076 -> 127.0.0.1:8885] +2017-06-20 18:03:18.681 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/tests/BasicEndToEndTest.cc(674)- Creating Subscriber +2017-06-20 18:03:18.682 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://prop/unit/ns/partition-testRoundRobinRoutingPolicy-partition-0, CONSUMER0, 0] Getting connection from pool +2017-06-20 18:03:18.683 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(155)- [persistent://prop/unit/ns/partition-testRoundRobinRoutingPolicy-partition-0, CONSUMER0, 0] Created consumer on broker [127.0.0.1:50076 -> 127.0.0.1:8885] +2017-06-20 18:03:18.683 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://prop/unit/ns/partition-testRoundRobinRoutingPolicy-partition-1, CONSUMER1, 1] Getting connection from pool +2017-06-20 18:03:18.684 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(155)- [persistent://prop/unit/ns/partition-testRoundRobinRoutingPolicy-partition-1, CONSUMER1, 1] Created consumer on broker [127.0.0.1:50076 -> 127.0.0.1:8885] +2017-06-20 18:03:18.685 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://prop/unit/ns/partition-testRoundRobinRoutingPolicy-partition-2, CONSUMER2, 2] Getting connection from pool +2017-06-20 18:03:18.685 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(155)- [persistent://prop/unit/ns/partition-testRoundRobinRoutingPolicy-partition-2, CONSUMER2, 2] Created consumer on broker [127.0.0.1:50076 -> 127.0.0.1:8885] +2017-06-20 18:03:18.686 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://prop/unit/ns/partition-testRoundRobinRoutingPolicy-partition-3, CONSUMER3, 3] Getting connection from pool +2017-06-20 18:03:18.686 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(155)- [persistent://prop/unit/ns/partition-testRoundRobinRoutingPolicy-partition-3, CONSUMER3, 3] Created consumer on broker [127.0.0.1:50076 -> 127.0.0.1:8885] +2017-06-20 18:03:18.686 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://prop/unit/ns/partition-testRoundRobinRoutingPolicy-partition-4, CONSUMER4, 4] Getting connection from pool +2017-06-20 18:03:18.687 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(155)- [persistent://prop/unit/ns/partition-testRoundRobinRoutingPolicy-partition-4, CONSUMER4, 4] Created consumer on broker [127.0.0.1:50076 -> 127.0.0.1:8885] +2017-06-20 18:03:18.706 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(639)- [persistent://prop/unit/ns/partition-testRoundRobinRoutingPolicy-partition-0, CONSUMER0, 0] Closed consumer 0 +2017-06-20 18:03:18.706 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(639)- [persistent://prop/unit/ns/partition-testRoundRobinRoutingPolicy-partition-1, CONSUMER1, 1] Closed consumer 1 +2017-06-20 18:03:18.707 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(639)- [persistent://prop/unit/ns/partition-testRoundRobinRoutingPolicy-partition-2, CONSUMER2, 2] Closed consumer 2 +2017-06-20 18:03:18.707 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(639)- [persistent://prop/unit/ns/partition-testRoundRobinRoutingPolicy-partition-3, CONSUMER3, 3] Closed consumer 3 +2017-06-20 18:03:18.707 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(639)- [persistent://prop/unit/ns/partition-testRoundRobinRoutingPolicy-partition-4, CONSUMER4, 4] Closed consumer 4 +2017-06-20 18:03:18.708 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ProducerImpl.cc(407)- [persistent://prop/unit/ns/partition-testRoundRobinRoutingPolicy-partition-0, standalone-0-367] Closed producer +2017-06-20 18:03:18.708 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ProducerImpl.cc(407)- [persistent://prop/unit/ns/partition-testRoundRobinRoutingPolicy-partition-1, standalone-0-368] Closed producer +2017-06-20 18:03:18.708 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ProducerImpl.cc(407)- [persistent://prop/unit/ns/partition-testRoundRobinRoutingPolicy-partition-2, standalone-0-369] Closed producer +2017-06-20 18:03:18.708 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ProducerImpl.cc(407)- [persistent://prop/unit/ns/partition-testRoundRobinRoutingPolicy-partition-3, standalone-0-370] Closed producer +2017-06-20 18:03:18.708 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ProducerImpl.cc(407)- [persistent://prop/unit/ns/partition-testRoundRobinRoutingPolicy-partition-4, standalone-0-371] Closed producer +2017-06-20 18:03:18.708 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(81)- [persistent://prop/unit/ns/partition-testRoundRobinRoutingPolicy-partition-4, CONSUMER4, 4] ~ConsumerImpl +2017-06-20 18:03:18.708 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(81)- [persistent://prop/unit/ns/partition-testRoundRobinRoutingPolicy-partition-3, CONSUMER3, 3] ~ConsumerImpl +2017-06-20 18:03:18.708 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(81)- [persistent://prop/unit/ns/partition-testRoundRobinRoutingPolicy-partition-2, CONSUMER2, 2] ~ConsumerImpl +2017-06-20 18:03:18.708 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(81)- [persistent://prop/unit/ns/partition-testRoundRobinRoutingPolicy-partition-1, CONSUMER1, 1] ~ConsumerImpl +2017-06-20 18:03:18.708 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(81)- [persistent://prop/unit/ns/partition-testRoundRobinRoutingPolicy-partition-0, CONSUMER0, 0] ~ConsumerImpl +2017-06-20 18:03:18.708 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientImpl.cc(84)- ~ClientImpl +[ OK ] BasicEndToEndTest.testRoundRobinRoutingPolicy (38 ms) +[ RUN ] BasicEndToEndTest.testMessageListener +{"reason":"Partitioned topic already exist"}2017-06-20 18:03:18.712 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/tests/BasicEndToEndTest.cc(727)- res = 409 +2017-06-20 18:03:18.713 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConnectionPool.cc(62)- Created connection for pulsar://localhost:8885 +2017-06-20 18:03:18.714 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientConnection.cc(265)- [127.0.0.1:50080 -> 127.0.0.1:8885] Connected to broker +2017-06-20 18:03:18.714 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://prop/unit/ns/partition-testMessageListener-partition-0, ] Getting connection from pool +2017-06-20 18:03:18.715 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://prop/unit/ns/partition-testMessageListener-partition-1, ] Getting connection from pool +2017-06-20 18:03:18.715 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://prop/unit/ns/partition-testMessageListener-partition-2, ] Getting connection from pool +2017-06-20 18:03:18.715 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://prop/unit/ns/partition-testMessageListener-partition-3, ] Getting connection from pool +2017-06-20 18:03:18.715 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://prop/unit/ns/partition-testMessageListener-partition-4, ] Getting connection from pool +2017-06-20 18:03:18.716 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ProducerImpl.cc(118)- [persistent://prop/unit/ns/partition-testMessageListener-partition-0, ] Created producer on broker [127.0.0.1:50080 -> 127.0.0.1:8885] +2017-06-20 18:03:18.717 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ProducerImpl.cc(118)- [persistent://prop/unit/ns/partition-testMessageListener-partition-1, ] Created producer on broker [127.0.0.1:50080 -> 127.0.0.1:8885] +2017-06-20 18:03:18.717 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ProducerImpl.cc(118)- [persistent://prop/unit/ns/partition-testMessageListener-partition-2, ] Created producer on broker [127.0.0.1:50080 -> 127.0.0.1:8885] +2017-06-20 18:03:18.718 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ProducerImpl.cc(118)- [persistent://prop/unit/ns/partition-testMessageListener-partition-3, ] Created producer on broker [127.0.0.1:50080 -> 127.0.0.1:8885] +2017-06-20 18:03:18.719 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ProducerImpl.cc(118)- [persistent://prop/unit/ns/partition-testMessageListener-partition-4, ] Created producer on broker [127.0.0.1:50080 -> 127.0.0.1:8885] +2017-06-20 18:03:18.719 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://prop/unit/ns/partition-testMessageListener-partition-0, subscription-A0, 0] Getting connection from pool +2017-06-20 18:03:18.719 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://prop/unit/ns/partition-testMessageListener-partition-1, subscription-A1, 1] Getting connection from pool +2017-06-20 18:03:18.719 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://prop/unit/ns/partition-testMessageListener-partition-2, subscription-A2, 2] Getting connection from pool +2017-06-20 18:03:18.719 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://prop/unit/ns/partition-testMessageListener-partition-3, subscription-A3, 3] Getting connection from pool +2017-06-20 18:03:18.719 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://prop/unit/ns/partition-testMessageListener-partition-4, subscription-A4, 4] Getting connection from pool +2017-06-20 18:03:18.720 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(155)- [persistent://prop/unit/ns/partition-testMessageListener-partition-0, subscription-A0, 0] Created consumer on broker [127.0.0.1:50080 -> 127.0.0.1:8885] +2017-06-20 18:03:18.720 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(155)- [persistent://prop/unit/ns/partition-testMessageListener-partition-1, subscription-A1, 1] Created consumer on broker [127.0.0.1:50080 -> 127.0.0.1:8885] +2017-06-20 18:03:18.721 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(155)- [persistent://prop/unit/ns/partition-testMessageListener-partition-2, subscription-A2, 2] Created consumer on broker [127.0.0.1:50080 -> 127.0.0.1:8885] +2017-06-20 18:03:18.721 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(155)- [persistent://prop/unit/ns/partition-testMessageListener-partition-3, subscription-A3, 3] Created consumer on broker [127.0.0.1:50080 -> 127.0.0.1:8885] +2017-06-20 18:03:18.721 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(155)- [persistent://prop/unit/ns/partition-testMessageListener-partition-4, subscription-A4, 4] Created consumer on broker [127.0.0.1:50080 -> 127.0.0.1:8885] +2017-06-20 18:03:18.721 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/PartitionedConsumerImpl.cc(222)- Successfully Subscribed to Partitioned Topic - persistent://prop/unit/ns/partition-testMessageListener with - 5 Partitions. +2017-06-20 18:03:23.738 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(639)- [persistent://prop/unit/ns/partition-testMessageListener-partition-0, subscription-A0, 0] Closed consumer 0 +2017-06-20 18:03:23.738 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(639)- [persistent://prop/unit/ns/partition-testMessageListener-partition-1, subscription-A1, 1] Closed consumer 1 +2017-06-20 18:03:23.738 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(639)- [persistent://prop/unit/ns/partition-testMessageListener-partition-2, subscription-A2, 2] Closed consumer 2 +2017-06-20 18:03:23.738 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(639)- [persistent://prop/unit/ns/partition-testMessageListener-partition-3, subscription-A3, 3] Closed consumer 3 +2017-06-20 18:03:23.739 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(639)- [persistent://prop/unit/ns/partition-testMessageListener-partition-4, subscription-A4, 4] Closed consumer 4 +2017-06-20 18:03:23.739 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ProducerImpl.cc(407)- [persistent://prop/unit/ns/partition-testMessageListener-partition-0, standalone-0-372] Closed producer +2017-06-20 18:03:23.739 : ERROR /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/PartitionedProducerImpl.cc(178)- Closing the producer failed for partition - 0 +[ OK ] BasicEndToEndTest.testMessageListener (5031 ms) +[ RUN ] BasicEndToEndTest.testMessageListenerPause +2017-06-20 18:03:23.739 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ProducerImpl.cc(407)- [persistent://prop/unit/ns/partition-testMessageListener-partition-1, standalone-0-373] Closed producer +2017-06-20 18:03:23.739 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ProducerImpl.cc(407)- [persistent://prop/unit/ns/partition-testMessageListener-partition-2, standalone-0-374] Closed producer +2017-06-20 18:03:23.740 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ProducerImpl.cc(407)- [persistent://prop/unit/ns/partition-testMessageListener-partition-3, standalone-0-375] Closed producer +2017-06-20 18:03:23.740 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ProducerImpl.cc(407)- [persistent://prop/unit/ns/partition-testMessageListener-partition-4, standalone-0-376] Closed producer +{"reason":"Partitioned topic already exist"}2017-06-20 18:03:23.744 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/tests/BasicEndToEndTest.cc(770)- res = 409 +2017-06-20 18:03:23.745 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConnectionPool.cc(62)- Created connection for pulsar://localhost:8885 +2017-06-20 18:03:23.746 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientConnection.cc(265)- [127.0.0.1:50100 -> 127.0.0.1:8885] Connected to broker +2017-06-20 18:03:23.746 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/partition-testMessageListener-pauses-partition-0, ] Getting connection from pool +2017-06-20 18:03:23.747 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/partition-testMessageListener-pauses-partition-1, ] Getting connection from pool +2017-06-20 18:03:23.747 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/partition-testMessageListener-pauses-partition-2, ] Getting connection from pool +2017-06-20 18:03:23.747 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/partition-testMessageListener-pauses-partition-3, ] Getting connection from pool +2017-06-20 18:03:23.747 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/partition-testMessageListener-pauses-partition-4, ] Getting connection from pool +2017-06-20 18:03:23.748 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ProducerImpl.cc(118)- [persistent://property/cluster/namespace/partition-testMessageListener-pauses-partition-0, ] Created producer on broker [127.0.0.1:50100 -> 127.0.0.1:8885] +2017-06-20 18:03:23.749 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ProducerImpl.cc(118)- [persistent://property/cluster/namespace/partition-testMessageListener-pauses-partition-1, ] Created producer on broker [127.0.0.1:50100 -> 127.0.0.1:8885] +2017-06-20 18:03:23.750 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ProducerImpl.cc(118)- [persistent://property/cluster/namespace/partition-testMessageListener-pauses-partition-2, ] Created producer on broker [127.0.0.1:50100 -> 127.0.0.1:8885] +2017-06-20 18:03:23.751 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ProducerImpl.cc(118)- [persistent://property/cluster/namespace/partition-testMessageListener-pauses-partition-3, ] Created producer on broker [127.0.0.1:50100 -> 127.0.0.1:8885] +2017-06-20 18:03:23.751 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ProducerImpl.cc(118)- [persistent://property/cluster/namespace/partition-testMessageListener-pauses-partition-4, ] Created producer on broker [127.0.0.1:50100 -> 127.0.0.1:8885] +2017-06-20 18:03:23.752 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/partition-testMessageListener-pauses-partition-0, subscription-name0, 0] Getting connection from pool +2017-06-20 18:03:23.752 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/partition-testMessageListener-pauses-partition-1, subscription-name1, 1] Getting connection from pool +2017-06-20 18:03:23.752 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/partition-testMessageListener-pauses-partition-2, subscription-name2, 2] Getting connection from pool +2017-06-20 18:03:23.752 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/partition-testMessageListener-pauses-partition-3, subscription-name3, 3] Getting connection from pool +2017-06-20 18:03:23.752 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/partition-testMessageListener-pauses-partition-4, subscription-name4, 4] Getting connection from pool +2017-06-20 18:03:23.753 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(155)- [persistent://property/cluster/namespace/partition-testMessageListener-pauses-partition-0, subscription-name0, 0] Created consumer on broker [127.0.0.1:50100 -> 127.0.0.1:8885] +2017-06-20 18:03:23.753 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(155)- [persistent://property/cluster/namespace/partition-testMessageListener-pauses-partition-1, subscription-name1, 1] Created consumer on broker [127.0.0.1:50100 -> 127.0.0.1:8885] +2017-06-20 18:03:23.753 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(155)- [persistent://property/cluster/namespace/partition-testMessageListener-pauses-partition-2, subscription-name2, 2] Created consumer on broker [127.0.0.1:50100 -> 127.0.0.1:8885] +2017-06-20 18:03:23.753 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(155)- [persistent://property/cluster/namespace/partition-testMessageListener-pauses-partition-3, subscription-name3, 3] Created consumer on broker [127.0.0.1:50100 -> 127.0.0.1:8885] +2017-06-20 18:03:23.754 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(155)- [persistent://property/cluster/namespace/partition-testMessageListener-pauses-partition-4, subscription-name4, 4] Created consumer on broker [127.0.0.1:50100 -> 127.0.0.1:8885] +2017-06-20 18:03:23.754 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/PartitionedConsumerImpl.cc(222)- Successfully Subscribed to Partitioned Topic - persistent://property/cluster/namespace/partition-testMessageListener-pauses with - 5 Partitions. +2017-06-20 18:03:23.754 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/PartitionedConsumerImpl.cc(102)- [persistent://property/cluster/namespace/partition-testMessageListener-pauses,subscription-name] Unsubscribing +2017-06-20 18:03:23.754 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(208)- [persistent://property/cluster/namespace/partition-testMessageListener-pauses-partition-0, subscription-name0, 0] Unsubscribing +2017-06-20 18:03:23.754 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(208)- [persistent://property/cluster/namespace/partition-testMessageListener-pauses-partition-1, subscription-name1, 1] Unsubscribing +2017-06-20 18:03:23.754 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(208)- [persistent://property/cluster/namespace/partition-testMessageListener-pauses-partition-2, subscription-name2, 2] Unsubscribing +2017-06-20 18:03:23.754 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(208)- [persistent://property/cluster/namespace/partition-testMessageListener-pauses-partition-3, subscription-name3, 3] Unsubscribing +2017-06-20 18:03:23.754 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(208)- [persistent://property/cluster/namespace/partition-testMessageListener-pauses-partition-4, subscription-name4, 4] Unsubscribing +2017-06-20 18:03:23.759 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(240)- [persistent://property/cluster/namespace/partition-testMessageListener-pauses-partition-0, subscription-name0, 0] Unsubscribed successfully +2017-06-20 18:03:23.759 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/PartitionedConsumerImpl.cc(146)- Successfully Unsubscribed Consumer - 0 for Subscription - subscription-name for Topic - persistent://property/cluster/namespace/partition-testMessageListener-pauses +2017-06-20 18:03:23.759 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(240)- [persistent://property/cluster/namespace/partition-testMessageListener-pauses-partition-1, subscription-name1, 1] Unsubscribed successfully +2017-06-20 18:03:23.759 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/PartitionedConsumerImpl.cc(146)- Successfully Unsubscribed Consumer - 1 for Subscription - subscription-name for Topic - persistent://property/cluster/namespace/partition-testMessageListener-pauses +2017-06-20 18:03:23.759 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(240)- [persistent://property/cluster/namespace/partition-testMessageListener-pauses-partition-2, subscription-name2, 2] Unsubscribed successfully +2017-06-20 18:03:23.759 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/PartitionedConsumerImpl.cc(146)- Successfully Unsubscribed Consumer - 2 for Subscription - subscription-name for Topic - persistent://property/cluster/namespace/partition-testMessageListener-pauses +2017-06-20 18:03:23.759 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(240)- [persistent://property/cluster/namespace/partition-testMessageListener-pauses-partition-3, subscription-name3, 3] Unsubscribed successfully +2017-06-20 18:03:23.759 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/PartitionedConsumerImpl.cc(146)- Successfully Unsubscribed Consumer - 3 for Subscription - subscription-name for Topic - persistent://property/cluster/namespace/partition-testMessageListener-pauses +2017-06-20 18:03:23.760 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(240)- [persistent://property/cluster/namespace/partition-testMessageListener-pauses-partition-4, subscription-name4, 4] Unsubscribed successfully +2017-06-20 18:03:23.760 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/PartitionedConsumerImpl.cc(146)- Successfully Unsubscribed Consumer - 4 for Subscription - subscription-name for Topic - persistent://property/cluster/namespace/partition-testMessageListener-pauses +2017-06-20 18:03:23.761 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/partition-testMessageListener-pauses-partition-0, subscription-name0, 5] Getting connection from pool +2017-06-20 18:03:23.761 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/partition-testMessageListener-pauses-partition-1, subscription-name1, 6] Getting connection from pool +2017-06-20 18:03:23.761 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/partition-testMessageListener-pauses-partition-2, subscription-name2, 7] Getting connection from pool +2017-06-20 18:03:23.761 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/partition-testMessageListener-pauses-partition-3, subscription-name3, 8] Getting connection from pool +2017-06-20 18:03:23.761 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/partition-testMessageListener-pauses-partition-4, subscription-name4, 9] Getting connection from pool +2017-06-20 18:03:23.765 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(155)- [persistent://property/cluster/namespace/partition-testMessageListener-pauses-partition-4, subscription-name4, 9] Created consumer on broker [127.0.0.1:50100 -> 127.0.0.1:8885] +2017-06-20 18:03:23.765 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(155)- [persistent://property/cluster/namespace/partition-testMessageListener-pauses-partition-2, subscription-name2, 7] Created consumer on broker [127.0.0.1:50100 -> 127.0.0.1:8885] +2017-06-20 18:03:23.766 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(155)- [persistent://property/cluster/namespace/partition-testMessageListener-pauses-partition-3, subscription-name3, 8] Created consumer on broker [127.0.0.1:50100 -> 127.0.0.1:8885] +2017-06-20 18:03:23.766 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(155)- [persistent://property/cluster/namespace/partition-testMessageListener-pauses-partition-0, subscription-name0, 5] Created consumer on broker [127.0.0.1:50100 -> 127.0.0.1:8885] +2017-06-20 18:03:23.766 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(155)- [persistent://property/cluster/namespace/partition-testMessageListener-pauses-partition-1, subscription-name1, 6] Created consumer on broker [127.0.0.1:50100 -> 127.0.0.1:8885] +2017-06-20 18:03:23.766 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/PartitionedConsumerImpl.cc(222)- Successfully Subscribed to Partitioned Topic - persistent://property/cluster/namespace/partition-testMessageListener-pauses with - 5 Partitions. +2017-06-20 18:04:19.879 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(639)- [persistent://property/cluster/namespace/partition-testMessageListener-pauses-partition-0, subscription-name0, 5] Closed consumer 5 +2017-06-20 18:04:19.880 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(639)- [persistent://property/cluster/namespace/partition-testMessageListener-pauses-partition-1, subscription-name1, 6] Closed consumer 6 +2017-06-20 18:04:19.880 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(639)- [persistent://property/cluster/namespace/partition-testMessageListener-pauses-partition-2, subscription-name2, 7] Closed consumer 7 +2017-06-20 18:04:19.881 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(639)- [persistent://property/cluster/namespace/partition-testMessageListener-pauses-partition-3, subscription-name3, 8] Closed consumer 8 +2017-06-20 18:04:19.882 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(639)- [persistent://property/cluster/namespace/partition-testMessageListener-pauses-partition-4, subscription-name4, 9] Closed consumer 9 +2017-06-20 18:04:19.882 : ERROR /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/PartitionedProducerImpl.cc(178)- Closing the producer failed for partition - 0 +[ OK ] BasicEndToEndTest.testMessageListenerPause (56143 ms) +[ RUN ] BasicEndToEndTest.testResendViaSendCallback +2017-06-20 18:04:19.882 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ProducerImpl.cc(407)- [persistent://property/cluster/namespace/partition-testMessageListener-pauses-partition-0, standalone-0-377] Closed producer +2017-06-20 18:04:19.882 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ProducerImpl.cc(407)- [persistent://property/cluster/namespace/partition-testMessageListener-pauses-partition-1, standalone-0-378] Closed producer +2017-06-20 18:04:19.883 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ProducerImpl.cc(407)- [persistent://property/cluster/namespace/partition-testMessageListener-pauses-partition-2, standalone-0-379] Closed producer +2017-06-20 18:04:19.883 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ProducerImpl.cc(407)- [persistent://property/cluster/namespace/partition-testMessageListener-pauses-partition-3, standalone-0-380] Closed producer +2017-06-20 18:04:19.883 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ProducerImpl.cc(407)- [persistent://property/cluster/namespace/partition-testMessageListener-pauses-partition-4, standalone-0-381] Closed producer +2017-06-20 18:04:19.883 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConnectionPool.cc(62)- Created connection for pulsar://localhost:8885 +2017-06-20 18:04:19.884 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientConnection.cc(265)- [127.0.0.1:50140 -> 127.0.0.1:8885] Connected to broker +2017-06-20 18:04:19.885 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://my-property/my-cluster/my-namespace/testResendViaListener, ] Getting connection from pool +2017-06-20 18:04:19.889 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ProducerImpl.cc(118)- [persistent://my-property/my-cluster/my-namespace/testResendViaListener, ] Created producer on broker [127.0.0.1:50140 -> 127.0.0.1:8885] +2017-06-20 18:04:23.213 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ProducerImpl.cc(407)- [persistent://my-property/my-cluster/my-namespace/testResendViaListener, standalone-0-382] Closed producer +2017-06-20 18:04:23.213 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientImpl.cc(84)- ~ClientImpl +[ OK ] BasicEndToEndTest.testResendViaSendCallback (3331 ms) +[ RUN ] BasicEndToEndTest.testStatsLatencies +2017-06-20 18:04:23.214 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConnectionPool.cc(62)- Created connection for pulsar://localhost:8885 +2017-06-20 18:04:23.215 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientConnection.cc(265)- [127.0.0.1:50142 -> 127.0.0.1:8885] Connected to broker +2017-06-20 18:04:23.217 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/testStatsLatencies, ] Getting connection from pool +2017-06-20 18:04:23.218 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ProducerImpl.cc(118)- [persistent://property/cluster/namespace/testStatsLatencies, ] Created producer on broker [127.0.0.1:50142 -> 127.0.0.1:8885] +2017-06-20 18:04:23.219 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/testStatsLatencies, subscription-name, 0] Getting connection from pool +2017-06-20 18:04:23.219 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(155)- [persistent://property/cluster/namespace/testStatsLatencies, subscription-name, 0] Created consumer on broker [127.0.0.1:50142 -> 127.0.0.1:8885] +2017-06-20 18:04:23.219 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(208)- [persistent://property/cluster/namespace/testStatsLatencies, subscription-name, 0] Unsubscribing +2017-06-20 18:04:23.222 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(240)- [persistent://property/cluster/namespace/testStatsLatencies, subscription-name, 0] Unsubscribed successfully +2017-06-20 18:04:23.222 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/testStatsLatencies, subscription-name, 1] Getting connection from pool +2017-06-20 18:04:23.226 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(155)- [persistent://property/cluster/namespace/testStatsLatencies, subscription-name, 1] Created consumer on broker [127.0.0.1:50142 -> 127.0.0.1:8885] +2017-06-20 18:04:28.217 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ProducerStatsImpl.cc(90)- Producer [persistent://property/cluster/namespace/testStatsLatencies, ] , ProducerStatsImpl (, numMsgsSent_ = 1000, numBytesSent_ = 12890, sendMap_ = {[Key: Ok, Value: 1000], }, latencyAccumulator_ = Latencies [ 50pct: 2104.95ms, 90pct: 3643.55ms, 99pct: 3902.63ms, 99.9pct: 3961.77ms], totalMsgsSent_ = 1000, totalBytesSent_ = 12890, totalAcksReceived_ = , totalSendMap_ = {[Key: Ok, Value: 1000], }, totalLatencyAccumulator_ = Latencies [ 50pct: 2104.95ms, 90pct: 3643.55ms, 99pct: 3902.63ms, 99.9pct: 3961.77ms]) +2017-06-20 18:04:28.219 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/testStatsLatencies, subscription-name, 0] , ConsumerStatsImpl (, numBytesRecieved_ = 0, totalNumBytesRecieved_ = 0, receivedMsgMap_ = {}, ackedMsgMap_ = {}, totalReceivedMsgMap_ = {}, totalAckedMsgMap_ = {}) +2017-06-20 18:04:28.223 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/testStatsLatencies, subscription-name, 1] , ConsumerStatsImpl (, numBytesRecieved_ = 0, totalNumBytesRecieved_ = 0, receivedMsgMap_ = {}, ackedMsgMap_ = {}, totalReceivedMsgMap_ = {}, totalAckedMsgMap_ = {}) +2017-06-20 18:04:33.221 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ProducerStatsImpl.cc(90)- Producer [persistent://property/cluster/namespace/testStatsLatencies, ] , ProducerStatsImpl (, numMsgsSent_ = 0, numBytesSent_ = 0, sendMap_ = {}, latencyAccumulator_ = Latencies [ 50pct: 0ms, 90pct: 0ms, 99pct: 0ms, 99.9pct: 0ms], totalMsgsSent_ = 1000, totalBytesSent_ = 12890, totalAcksReceived_ = , totalSendMap_ = {[Key: Ok, Value: 1000], }, totalLatencyAccumulator_ = Latencies [ 50pct: 2104.95ms, 90pct: 3643.55ms, 99pct: 3902.63ms, 99.9pct: 3961.77ms]) +2017-06-20 18:04:33.221 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/testStatsLatencies, subscription-name, 0] , ConsumerStatsImpl (, numBytesRecieved_ = 0, totalNumBytesRecieved_ = 0, receivedMsgMap_ = {}, ackedMsgMap_ = {}, totalReceivedMsgMap_ = {}, totalAckedMsgMap_ = {}) +2017-06-20 18:04:33.223 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/testStatsLatencies, subscription-name, 1] , ConsumerStatsImpl (, numBytesRecieved_ = 12890, totalNumBytesRecieved_ = 12890, receivedMsgMap_ = {[Key: Ok, Value: 1000], }, ackedMsgMap_ = {[Key: {Result: Ok, ackType0}, Value: 1000], }, totalReceivedMsgMap_ = {[Key: Ok, Value: 1000], }, totalAckedMsgMap_ = {[Key: {Result: Ok, ackType0}, Value: 1000], }) +2017-06-20 18:04:34.258 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(81)- [persistent://property/cluster/namespace/testStatsLatencies, subscription-name, 0] ~ConsumerImpl +2017-06-20 18:04:34.258 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(81)- [persistent://property/cluster/namespace/testStatsLatencies, subscription-name, 1] ~ConsumerImpl +2017-06-20 18:04:34.258 : WARN /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(84)- [persistent://property/cluster/namespace/testStatsLatencies, subscription-name, 1] Destroyed consumer which was not properly closed +2017-06-20 18:04:34.258 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientImpl.cc(84)- ~ClientImpl +2017-06-20 18:04:34.258 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ProducerImpl.cc(75)- [persistent://property/cluster/namespace/testStatsLatencies, standalone-0-383] ~ProducerImpl +2017-06-20 18:04:34.258 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ProducerImpl.cc(355)- Producer - [persistent://property/cluster/namespace/testStatsLatencies, standalone-0-383] , [batching = off] +[ OK ] BasicEndToEndTest.testStatsLatencies (11046 ms) +[----------] 25 tests from BasicEndToEndTest (85665 ms total) + +[----------] 9 tests from BatchMessageTest +[ RUN ] BatchMessageTest.testProducerConfig +[ OK ] BatchMessageTest.testProducerConfig (3 ms) +[ RUN ] BatchMessageTest.testProducerTimeout +2017-06-20 18:04:34.263 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConnectionPool.cc(62)- Created connection for pulsar://localhost:8885 +2017-06-20 18:04:34.265 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientConnection.cc(265)- [127.0.0.1:50146 -> 127.0.0.1:8885] Connected to broker +2017-06-20 18:04:34.266 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/BatchMessageContainer.cc(39)- { BatchContainer [size = 0] [batchSizeInBytes_ = 0] [maxAllowedMessageBatchSizeInBytes_ = 131072] [maxAllowedNumMessagesInBatch_ = 3] [topicName = persistent://property/cluster/namespace/1498006988testProducerTimeout] [producerName_ = ] [batchSizeInBytes_ = 0] [numberOfBatchesSent = 0] [averageBatchSize = 0]} BatchMessageContainer constructed +2017-06-20 18:04:34.266 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/1498006988testProducerTimeout, ] Getting connection from pool +2017-06-20 18:04:34.270 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ProducerImpl.cc(118)- [persistent://property/cluster/namespace/1498006988testProducerTimeout, ] Created producer on broker [127.0.0.1:50146 -> 127.0.0.1:8885] +2017-06-20 18:04:34.271 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/1498006988testProducerTimeout, subscription-name, 0] Getting connection from pool +2017-06-20 18:04:34.274 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(155)- [persistent://property/cluster/namespace/1498006988testProducerTimeout, subscription-name, 0] Created consumer on broker [127.0.0.1:50146 -> 127.0.0.1:8885] +2017-06-20 18:04:34.275 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(639)- [persistent://property/cluster/namespace/1498006988testProducerTimeout, subscription-name, 0] Closed consumer 0 +2017-06-20 18:04:34.276 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/1498006988testProducerTimeout, subscription-name, 1] Getting connection from pool +2017-06-20 18:04:34.276 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(155)- [persistent://property/cluster/namespace/1498006988testProducerTimeout, subscription-name, 1] Created consumer on broker [127.0.0.1:50146 -> 127.0.0.1:8885] +2017-06-20 18:04:35.270 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ProducerStatsImpl.cc(90)- Producer [persistent://property/cluster/namespace/1498006988testProducerTimeout, ] , ProducerStatsImpl (, numMsgsSent_ = 1, numBytesSent_ = 32, sendMap_ = {}, latencyAccumulator_ = Latencies [ 50pct: 0ms, 90pct: 0ms, 99pct: 0ms, 99.9pct: 0ms], totalMsgsSent_ = 1, totalBytesSent_ = 32, totalAcksReceived_ = , totalSendMap_ = {}, totalLatencyAccumulator_ = Latencies [ 50pct: 0ms, 90pct: 0ms, 99pct: 0ms, 99.9pct: 0ms]) +2017-06-20 18:04:35.271 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/1498006988testProducerTimeout, subscription-name, 0] , ConsumerStatsImpl (, numBytesRecieved_ = 0, totalNumBytesRecieved_ = 0, receivedMsgMap_ = {}, ackedMsgMap_ = {}, totalReceivedMsgMap_ = {}, totalAckedMsgMap_ = {}) +2017-06-20 18:04:35.276 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/1498006988testProducerTimeout, subscription-name, 1] , ConsumerStatsImpl (, numBytesRecieved_ = 0, totalNumBytesRecieved_ = 0, receivedMsgMap_ = {}, ackedMsgMap_ = {}, totalReceivedMsgMap_ = {}, totalAckedMsgMap_ = {}) +2017-06-20 18:04:36.274 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ProducerStatsImpl.cc(90)- Producer [persistent://property/cluster/namespace/1498006988testProducerTimeout, ] , ProducerStatsImpl (, numMsgsSent_ = 0, numBytesSent_ = 0, sendMap_ = {}, latencyAccumulator_ = Latencies [ 50pct: 0ms, 90pct: 0ms, 99pct: 0ms, 99.9pct: 0ms], totalMsgsSent_ = 1, totalBytesSent_ = 32, totalAcksReceived_ = , totalSendMap_ = {}, totalLatencyAccumulator_ = Latencies [ 50pct: 0ms, 90pct: 0ms, 99pct: 0ms, 99.9pct: 0ms]) +2017-06-20 18:04:36.274 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/1498006988testProducerTimeout, subscription-name, 0] , ConsumerStatsImpl (, numBytesRecieved_ = 0, totalNumBytesRecieved_ = 0, receivedMsgMap_ = {}, ackedMsgMap_ = {}, totalReceivedMsgMap_ = {}, totalAckedMsgMap_ = {}) +2017-06-20 18:04:36.276 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/1498006988testProducerTimeout, subscription-name, 1] , ConsumerStatsImpl (, numBytesRecieved_ = 0, totalNumBytesRecieved_ = 0, receivedMsgMap_ = {}, ackedMsgMap_ = {}, totalReceivedMsgMap_ = {}, totalAckedMsgMap_ = {}) +2017-06-20 18:04:37.278 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ProducerStatsImpl.cc(90)- Producer [persistent://property/cluster/namespace/1498006988testProducerTimeout, ] , ProducerStatsImpl (, numMsgsSent_ = 0, numBytesSent_ = 0, sendMap_ = {}, latencyAccumulator_ = Latencies [ 50pct: 0ms, 90pct: 0ms, 99pct: 0ms, 99.9pct: 0ms], totalMsgsSent_ = 1, totalBytesSent_ = 32, totalAcksReceived_ = , totalSendMap_ = {}, totalLatencyAccumulator_ = Latencies [ 50pct: 0ms, 90pct: 0ms, 99pct: 0ms, 99.9pct: 0ms]) +2017-06-20 18:04:37.278 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/1498006988testProducerTimeout, subscription-name, 0] , ConsumerStatsImpl (, numBytesRecieved_ = 0, totalNumBytesRecieved_ = 0, receivedMsgMap_ = {}, ackedMsgMap_ = {}, totalReceivedMsgMap_ = {}, totalAckedMsgMap_ = {}) +2017-06-20 18:04:37.278 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/1498006988testProducerTimeout, subscription-name, 1] , ConsumerStatsImpl (, numBytesRecieved_ = 0, totalNumBytesRecieved_ = 0, receivedMsgMap_ = {}, ackedMsgMap_ = {}, totalReceivedMsgMap_ = {}, totalAckedMsgMap_ = {}) +2017-06-20 18:04:38.280 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ProducerStatsImpl.cc(90)- Producer [persistent://property/cluster/namespace/1498006988testProducerTimeout, ] , ProducerStatsImpl (, numMsgsSent_ = 0, numBytesSent_ = 0, sendMap_ = {}, latencyAccumulator_ = Latencies [ 50pct: 0ms, 90pct: 0ms, 99pct: 0ms, 99.9pct: 0ms], totalMsgsSent_ = 1, totalBytesSent_ = 32, totalAcksReceived_ = , totalSendMap_ = {}, totalLatencyAccumulator_ = Latencies [ 50pct: 0ms, 90pct: 0ms, 99pct: 0ms, 99.9pct: 0ms]) +2017-06-20 18:04:38.280 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/1498006988testProducerTimeout, subscription-name, 0] , ConsumerStatsImpl (, numBytesRecieved_ = 0, totalNumBytesRecieved_ = 0, receivedMsgMap_ = {}, ackedMsgMap_ = {}, totalReceivedMsgMap_ = {}, totalAckedMsgMap_ = {}) +2017-06-20 18:04:38.280 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/1498006988testProducerTimeout, subscription-name, 1] , ConsumerStatsImpl (, numBytesRecieved_ = 0, totalNumBytesRecieved_ = 0, receivedMsgMap_ = {}, ackedMsgMap_ = {}, totalReceivedMsgMap_ = {}, totalAckedMsgMap_ = {}) +2017-06-20 18:04:39.281 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ProducerStatsImpl.cc(90)- Producer [persistent://property/cluster/namespace/1498006988testProducerTimeout, ] , ProducerStatsImpl (, numMsgsSent_ = 1, numBytesSent_ = 32, sendMap_ = {[Key: Ok, Value: 1], }, latencyAccumulator_ = Latencies [ 50pct: 0ms, 90pct: 0ms, 99pct: 0ms, 99.9pct: 0ms], totalMsgsSent_ = 2, totalBytesSent_ = 64, totalAcksReceived_ = , totalSendMap_ = {[Key: Ok, Value: 1], }, totalLatencyAccumulator_ = Latencies [ 50pct: 0ms, 90pct: 0ms, 99pct: 0ms, 99.9pct: 0ms]) +2017-06-20 18:04:39.281 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/1498006988testProducerTimeout, subscription-name, 0] , ConsumerStatsImpl (, numBytesRecieved_ = 0, totalNumBytesRecieved_ = 0, receivedMsgMap_ = {}, ackedMsgMap_ = {}, totalReceivedMsgMap_ = {}, totalAckedMsgMap_ = {}) +2017-06-20 18:04:39.281 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/1498006988testProducerTimeout, subscription-name, 1] , ConsumerStatsImpl (, numBytesRecieved_ = 0, totalNumBytesRecieved_ = 0, receivedMsgMap_ = {}, ackedMsgMap_ = {}, totalReceivedMsgMap_ = {}, totalAckedMsgMap_ = {}) +2017-06-20 18:04:40.281 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ProducerStatsImpl.cc(90)- Producer [persistent://property/cluster/namespace/1498006988testProducerTimeout, ] , ProducerStatsImpl (, numMsgsSent_ = 0, numBytesSent_ = 0, sendMap_ = {}, latencyAccumulator_ = Latencies [ 50pct: 0ms, 90pct: 0ms, 99pct: 0ms, 99.9pct: 0ms], totalMsgsSent_ = 2, totalBytesSent_ = 64, totalAcksReceived_ = , totalSendMap_ = {[Key: Ok, Value: 1], }, totalLatencyAccumulator_ = Latencies [ 50pct: 0ms, 90pct: 0ms, 99pct: 0ms, 99.9pct: 0ms]) +2017-06-20 18:04:40.281 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/1498006988testProducerTimeout, subscription-name, 0] , ConsumerStatsImpl (, numBytesRecieved_ = 0, totalNumBytesRecieved_ = 0, receivedMsgMap_ = {}, ackedMsgMap_ = {}, totalReceivedMsgMap_ = {}, totalAckedMsgMap_ = {}) +2017-06-20 18:04:40.281 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/1498006988testProducerTimeout, subscription-name, 1] , ConsumerStatsImpl (, numBytesRecieved_ = 0, totalNumBytesRecieved_ = 0, receivedMsgMap_ = {}, ackedMsgMap_ = {}, totalReceivedMsgMap_ = {}, totalAckedMsgMap_ = {}) +2017-06-20 18:04:41.282 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ProducerStatsImpl.cc(90)- Producer [persistent://property/cluster/namespace/1498006988testProducerTimeout, ] , ProducerStatsImpl (, numMsgsSent_ = 0, numBytesSent_ = 0, sendMap_ = {}, latencyAccumulator_ = Latencies [ 50pct: 0ms, 90pct: 0ms, 99pct: 0ms, 99.9pct: 0ms], totalMsgsSent_ = 2, totalBytesSent_ = 64, totalAcksReceived_ = , totalSendMap_ = {[Key: Ok, Value: 1], }, totalLatencyAccumulator_ = Latencies [ 50pct: 0ms, 90pct: 0ms, 99pct: 0ms, 99.9pct: 0ms]) +2017-06-20 18:04:41.282 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/1498006988testProducerTimeout, subscription-name, 0] , ConsumerStatsImpl (, numBytesRecieved_ = 0, totalNumBytesRecieved_ = 0, receivedMsgMap_ = {}, ackedMsgMap_ = {}, totalReceivedMsgMap_ = {}, totalAckedMsgMap_ = {}) +2017-06-20 18:04:41.282 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/1498006988testProducerTimeout, subscription-name, 1] , ConsumerStatsImpl (, numBytesRecieved_ = 0, totalNumBytesRecieved_ = 0, receivedMsgMap_ = {}, ackedMsgMap_ = {}, totalReceivedMsgMap_ = {}, totalAckedMsgMap_ = {}) +2017-06-20 18:04:42.282 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ProducerStatsImpl.cc(90)- Producer [persistent://property/cluster/namespace/1498006988testProducerTimeout, ] , ProducerStatsImpl (, numMsgsSent_ = 0, numBytesSent_ = 0, sendMap_ = {}, latencyAccumulator_ = Latencies [ 50pct: 0ms, 90pct: 0ms, 99pct: 0ms, 99.9pct: 0ms], totalMsgsSent_ = 2, totalBytesSent_ = 64, totalAcksReceived_ = , totalSendMap_ = {[Key: Ok, Value: 1], }, totalLatencyAccumulator_ = Latencies [ 50pct: 0ms, 90pct: 0ms, 99pct: 0ms, 99.9pct: 0ms]) +2017-06-20 18:04:42.282 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/1498006988testProducerTimeout, subscription-name, 0] , ConsumerStatsImpl (, numBytesRecieved_ = 0, totalNumBytesRecieved_ = 0, receivedMsgMap_ = {}, ackedMsgMap_ = {}, totalReceivedMsgMap_ = {}, totalAckedMsgMap_ = {}) +2017-06-20 18:04:42.282 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/1498006988testProducerTimeout, subscription-name, 1] , ConsumerStatsImpl (, numBytesRecieved_ = 0, totalNumBytesRecieved_ = 0, receivedMsgMap_ = {}, ackedMsgMap_ = {}, totalReceivedMsgMap_ = {}, totalAckedMsgMap_ = {}) +2017-06-20 18:04:43.282 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ProducerStatsImpl.cc(90)- Producer [persistent://property/cluster/namespace/1498006988testProducerTimeout, ] , ProducerStatsImpl (, numMsgsSent_ = 1, numBytesSent_ = 32, sendMap_ = {[Key: Ok, Value: 1], }, latencyAccumulator_ = Latencies [ 50pct: 0ms, 90pct: 0ms, 99pct: 0ms, 99.9pct: 0ms], totalMsgsSent_ = 3, totalBytesSent_ = 96, totalAcksReceived_ = , totalSendMap_ = {[Key: Ok, Value: 2], }, totalLatencyAccumulator_ = Latencies [ 50pct: 0ms, 90pct: 0ms, 99pct: 0ms, 99.9pct: 0ms]) +2017-06-20 18:04:43.282 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/1498006988testProducerTimeout, subscription-name, 0] , ConsumerStatsImpl (, numBytesRecieved_ = 0, totalNumBytesRecieved_ = 0, receivedMsgMap_ = {}, ackedMsgMap_ = {}, totalReceivedMsgMap_ = {}, totalAckedMsgMap_ = {}) +2017-06-20 18:04:43.282 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/1498006988testProducerTimeout, subscription-name, 1] , ConsumerStatsImpl (, numBytesRecieved_ = 0, totalNumBytesRecieved_ = 0, receivedMsgMap_ = {}, ackedMsgMap_ = {}, totalReceivedMsgMap_ = {}, totalAckedMsgMap_ = {}) +2017-06-20 18:04:44.285 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ProducerStatsImpl.cc(90)- Producer [persistent://property/cluster/namespace/1498006988testProducerTimeout, ] , ProducerStatsImpl (, numMsgsSent_ = 0, numBytesSent_ = 0, sendMap_ = {}, latencyAccumulator_ = Latencies [ 50pct: 0ms, 90pct: 0ms, 99pct: 0ms, 99.9pct: 0ms], totalMsgsSent_ = 3, totalBytesSent_ = 96, totalAcksReceived_ = , totalSendMap_ = {[Key: Ok, Value: 2], }, totalLatencyAccumulator_ = Latencies [ 50pct: 0ms, 90pct: 0ms, 99pct: 0ms, 99.9pct: 0ms]) +2017-06-20 18:04:44.285 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/1498006988testProducerTimeout, subscription-name, 0] , ConsumerStatsImpl (, numBytesRecieved_ = 0, totalNumBytesRecieved_ = 0, receivedMsgMap_ = {}, ackedMsgMap_ = {}, totalReceivedMsgMap_ = {}, totalAckedMsgMap_ = {}) +2017-06-20 18:04:44.285 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/1498006988testProducerTimeout, subscription-name, 1] , ConsumerStatsImpl (, numBytesRecieved_ = 0, totalNumBytesRecieved_ = 0, receivedMsgMap_ = {}, ackedMsgMap_ = {}, totalReceivedMsgMap_ = {}, totalAckedMsgMap_ = {}) +2017-06-20 18:04:45.285 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ProducerStatsImpl.cc(90)- Producer [persistent://property/cluster/namespace/1498006988testProducerTimeout, ] , ProducerStatsImpl (, numMsgsSent_ = 0, numBytesSent_ = 0, sendMap_ = {}, latencyAccumulator_ = Latencies [ 50pct: 0ms, 90pct: 0ms, 99pct: 0ms, 99.9pct: 0ms], totalMsgsSent_ = 3, totalBytesSent_ = 96, totalAcksReceived_ = , totalSendMap_ = {[Key: Ok, Value: 2], }, totalLatencyAccumulator_ = Latencies [ 50pct: 0ms, 90pct: 0ms, 99pct: 0ms, 99.9pct: 0ms]) +2017-06-20 18:04:45.285 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/1498006988testProducerTimeout, subscription-name, 0] , ConsumerStatsImpl (, numBytesRecieved_ = 0, totalNumBytesRecieved_ = 0, receivedMsgMap_ = {}, ackedMsgMap_ = {}, totalReceivedMsgMap_ = {}, totalAckedMsgMap_ = {}) +2017-06-20 18:04:45.285 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/1498006988testProducerTimeout, subscription-name, 1] , ConsumerStatsImpl (, numBytesRecieved_ = 0, totalNumBytesRecieved_ = 0, receivedMsgMap_ = {}, ackedMsgMap_ = {}, totalReceivedMsgMap_ = {}, totalAckedMsgMap_ = {}) +2017-06-20 18:04:46.288 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ProducerStatsImpl.cc(90)- Producer [persistent://property/cluster/namespace/1498006988testProducerTimeout, ] , ProducerStatsImpl (, numMsgsSent_ = 0, numBytesSent_ = 0, sendMap_ = {}, latencyAccumulator_ = Latencies [ 50pct: 0ms, 90pct: 0ms, 99pct: 0ms, 99.9pct: 0ms], totalMsgsSent_ = 3, totalBytesSent_ = 96, totalAcksReceived_ = , totalSendMap_ = {[Key: Ok, Value: 2], }, totalLatencyAccumulator_ = Latencies [ 50pct: 0ms, 90pct: 0ms, 99pct: 0ms, 99.9pct: 0ms]) +2017-06-20 18:04:46.288 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/1498006988testProducerTimeout, subscription-name, 0] , ConsumerStatsImpl (, numBytesRecieved_ = 0, totalNumBytesRecieved_ = 0, receivedMsgMap_ = {}, ackedMsgMap_ = {}, totalReceivedMsgMap_ = {}, totalAckedMsgMap_ = {}) +2017-06-20 18:04:46.288 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/1498006988testProducerTimeout, subscription-name, 1] , ConsumerStatsImpl (, numBytesRecieved_ = 0, totalNumBytesRecieved_ = 0, receivedMsgMap_ = {}, ackedMsgMap_ = {}, totalReceivedMsgMap_ = {}, totalAckedMsgMap_ = {}) +2017-06-20 18:04:47.292 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ProducerStatsImpl.cc(90)- Producer [persistent://property/cluster/namespace/1498006988testProducerTimeout, ] , ProducerStatsImpl (, numMsgsSent_ = 1, numBytesSent_ = 32, sendMap_ = {[Key: Ok, Value: 1], }, latencyAccumulator_ = Latencies [ 50pct: 0ms, 90pct: 0ms, 99pct: 0ms, 99.9pct: 0ms], totalMsgsSent_ = 4, totalBytesSent_ = 128, totalAcksReceived_ = , totalSendMap_ = {[Key: Ok, Value: 3], }, totalLatencyAccumulator_ = Latencies [ 50pct: 4004.62ms, 90pct: 0ms, 99pct: 0ms, 99.9pct: 0ms]) +2017-06-20 18:04:47.292 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/1498006988testProducerTimeout, subscription-name, 0] , ConsumerStatsImpl (, numBytesRecieved_ = 0, totalNumBytesRecieved_ = 0, receivedMsgMap_ = {}, ackedMsgMap_ = {}, totalReceivedMsgMap_ = {}, totalAckedMsgMap_ = {}) +2017-06-20 18:04:47.292 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/1498006988testProducerTimeout, subscription-name, 1] , ConsumerStatsImpl (, numBytesRecieved_ = 0, totalNumBytesRecieved_ = 0, receivedMsgMap_ = {}, ackedMsgMap_ = {}, totalReceivedMsgMap_ = {}, totalAckedMsgMap_ = {}) +2017-06-20 18:04:48.296 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ProducerStatsImpl.cc(90)- Producer [persistent://property/cluster/namespace/1498006988testProducerTimeout, ] , ProducerStatsImpl (, numMsgsSent_ = 0, numBytesSent_ = 0, sendMap_ = {}, latencyAccumulator_ = Latencies [ 50pct: 0ms, 90pct: 0ms, 99pct: 0ms, 99.9pct: 0ms], totalMsgsSent_ = 4, totalBytesSent_ = 128, totalAcksReceived_ = , totalSendMap_ = {[Key: Ok, Value: 3], }, totalLatencyAccumulator_ = Latencies [ 50pct: 4004.62ms, 90pct: 0ms, 99pct: 0ms, 99.9pct: 0ms]) +2017-06-20 18:04:48.296 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/1498006988testProducerTimeout, subscription-name, 0] , ConsumerStatsImpl (, numBytesRecieved_ = 0, totalNumBytesRecieved_ = 0, receivedMsgMap_ = {}, ackedMsgMap_ = {}, totalReceivedMsgMap_ = {}, totalAckedMsgMap_ = {}) +2017-06-20 18:04:48.296 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/1498006988testProducerTimeout, subscription-name, 1] , ConsumerStatsImpl (, numBytesRecieved_ = 0, totalNumBytesRecieved_ = 0, receivedMsgMap_ = {}, ackedMsgMap_ = {}, totalReceivedMsgMap_ = {}, totalAckedMsgMap_ = {}) +2017-06-20 18:04:49.300 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ProducerStatsImpl.cc(90)- Producer [persistent://property/cluster/namespace/1498006988testProducerTimeout, ] , ProducerStatsImpl (, numMsgsSent_ = 0, numBytesSent_ = 0, sendMap_ = {}, latencyAccumulator_ = Latencies [ 50pct: 0ms, 90pct: 0ms, 99pct: 0ms, 99.9pct: 0ms], totalMsgsSent_ = 4, totalBytesSent_ = 128, totalAcksReceived_ = , totalSendMap_ = {[Key: Ok, Value: 3], }, totalLatencyAccumulator_ = Latencies [ 50pct: 4004.62ms, 90pct: 0ms, 99pct: 0ms, 99.9pct: 0ms]) +2017-06-20 18:04:49.300 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/1498006988testProducerTimeout, subscription-name, 0] , ConsumerStatsImpl (, numBytesRecieved_ = 0, totalNumBytesRecieved_ = 0, receivedMsgMap_ = {}, ackedMsgMap_ = {}, totalReceivedMsgMap_ = {}, totalAckedMsgMap_ = {}) +2017-06-20 18:04:49.300 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/1498006988testProducerTimeout, subscription-name, 1] , ConsumerStatsImpl (, numBytesRecieved_ = 0, totalNumBytesRecieved_ = 0, receivedMsgMap_ = {}, ackedMsgMap_ = {}, totalReceivedMsgMap_ = {}, totalAckedMsgMap_ = {}) +2017-06-20 18:04:50.300 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ProducerStatsImpl.cc(90)- Producer [persistent://property/cluster/namespace/1498006988testProducerTimeout, ] , ProducerStatsImpl (, numMsgsSent_ = 0, numBytesSent_ = 0, sendMap_ = {[Key: Ok, Value: 1], }, latencyAccumulator_ = Latencies [ 50pct: 0ms, 90pct: 0ms, 99pct: 0ms, 99.9pct: 0ms], totalMsgsSent_ = 4, totalBytesSent_ = 128, totalAcksReceived_ = , totalSendMap_ = {[Key: Ok, Value: 4], }, totalLatencyAccumulator_ = Latencies [ 50pct: 4004.62ms, 90pct: 0ms, 99pct: 0ms, 99.9pct: 0ms]) +2017-06-20 18:04:50.300 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/1498006988testProducerTimeout, subscription-name, 0] , ConsumerStatsImpl (, numBytesRecieved_ = 0, totalNumBytesRecieved_ = 0, receivedMsgMap_ = {}, ackedMsgMap_ = {}, totalReceivedMsgMap_ = {}, totalAckedMsgMap_ = {}) +2017-06-20 18:04:50.300 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/1498006988testProducerTimeout, subscription-name, 1] , ConsumerStatsImpl (, numBytesRecieved_ = 128, totalNumBytesRecieved_ = 128, receivedMsgMap_ = {[Key: Ok, Value: 4], }, ackedMsgMap_ = {[Key: {Result: Ok, ackType0}, Value: 4], }, totalReceivedMsgMap_ = {[Key: Ok, Value: 4], }, totalAckedMsgMap_ = {[Key: {Result: Ok, ackType0}, Value: 4], }) +2017-06-20 18:04:51.304 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ProducerStatsImpl.cc(90)- Producer [persistent://property/cluster/namespace/1498006988testProducerTimeout, ] , ProducerStatsImpl (, numMsgsSent_ = 0, numBytesSent_ = 0, sendMap_ = {}, latencyAccumulator_ = Latencies [ 50pct: 0ms, 90pct: 0ms, 99pct: 0ms, 99.9pct: 0ms], totalMsgsSent_ = 4, totalBytesSent_ = 128, totalAcksReceived_ = , totalSendMap_ = {[Key: Ok, Value: 4], }, totalLatencyAccumulator_ = Latencies [ 50pct: 4004.62ms, 90pct: 0ms, 99pct: 0ms, 99.9pct: 0ms]) +2017-06-20 18:04:51.304 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/1498006988testProducerTimeout, subscription-name, 0] , ConsumerStatsImpl (, numBytesRecieved_ = 0, totalNumBytesRecieved_ = 0, receivedMsgMap_ = {}, ackedMsgMap_ = {}, totalReceivedMsgMap_ = {}, totalAckedMsgMap_ = {}) +2017-06-20 18:04:51.304 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/1498006988testProducerTimeout, subscription-name, 1] , ConsumerStatsImpl (, numBytesRecieved_ = 0, totalNumBytesRecieved_ = 128, receivedMsgMap_ = {}, ackedMsgMap_ = {}, totalReceivedMsgMap_ = {[Key: Ok, Value: 4], }, totalAckedMsgMap_ = {[Key: {Result: Ok, ackType0}, Value: 4], }) +2017-06-20 18:04:52.304 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ProducerStatsImpl.cc(90)- Producer [persistent://property/cluster/namespace/1498006988testProducerTimeout, ] , ProducerStatsImpl (, numMsgsSent_ = 0, numBytesSent_ = 0, sendMap_ = {}, latencyAccumulator_ = Latencies [ 50pct: 0ms, 90pct: 0ms, 99pct: 0ms, 99.9pct: 0ms], totalMsgsSent_ = 4, totalBytesSent_ = 128, totalAcksReceived_ = , totalSendMap_ = {[Key: Ok, Value: 4], }, totalLatencyAccumulator_ = Latencies [ 50pct: 4004.62ms, 90pct: 0ms, 99pct: 0ms, 99.9pct: 0ms]) +2017-06-20 18:04:52.305 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/1498006988testProducerTimeout, subscription-name, 0] , ConsumerStatsImpl (, numBytesRecieved_ = 0, totalNumBytesRecieved_ = 0, receivedMsgMap_ = {}, ackedMsgMap_ = {}, totalReceivedMsgMap_ = {}, totalAckedMsgMap_ = {}) +2017-06-20 18:04:52.305 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/1498006988testProducerTimeout, subscription-name, 1] , ConsumerStatsImpl (, numBytesRecieved_ = 0, totalNumBytesRecieved_ = 128, receivedMsgMap_ = {}, ackedMsgMap_ = {}, totalReceivedMsgMap_ = {[Key: Ok, Value: 4], }, totalAckedMsgMap_ = {[Key: {Result: Ok, ackType0}, Value: 4], }) +2017-06-20 18:04:53.307 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ProducerStatsImpl.cc(90)- Producer [persistent://property/cluster/namespace/1498006988testProducerTimeout, ] , ProducerStatsImpl (, numMsgsSent_ = 0, numBytesSent_ = 0, sendMap_ = {}, latencyAccumulator_ = Latencies [ 50pct: 0ms, 90pct: 0ms, 99pct: 0ms, 99.9pct: 0ms], totalMsgsSent_ = 4, totalBytesSent_ = 128, totalAcksReceived_ = , totalSendMap_ = {[Key: Ok, Value: 4], }, totalLatencyAccumulator_ = Latencies [ 50pct: 4004.62ms, 90pct: 0ms, 99pct: 0ms, 99.9pct: 0ms]) +2017-06-20 18:04:53.308 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/1498006988testProducerTimeout, subscription-name, 0] , ConsumerStatsImpl (, numBytesRecieved_ = 0, totalNumBytesRecieved_ = 0, receivedMsgMap_ = {}, ackedMsgMap_ = {}, totalReceivedMsgMap_ = {}, totalAckedMsgMap_ = {}) +2017-06-20 18:04:53.308 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/1498006988testProducerTimeout, subscription-name, 1] , ConsumerStatsImpl (, numBytesRecieved_ = 0, totalNumBytesRecieved_ = 128, receivedMsgMap_ = {}, ackedMsgMap_ = {}, totalReceivedMsgMap_ = {[Key: Ok, Value: 4], }, totalAckedMsgMap_ = {[Key: {Result: Ok, ackType0}, Value: 4], }) +2017-06-20 18:04:54.310 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ProducerStatsImpl.cc(90)- Producer [persistent://property/cluster/namespace/1498006988testProducerTimeout, ] , ProducerStatsImpl (, numMsgsSent_ = 0, numBytesSent_ = 0, sendMap_ = {}, latencyAccumulator_ = Latencies [ 50pct: 0ms, 90pct: 0ms, 99pct: 0ms, 99.9pct: 0ms], totalMsgsSent_ = 4, totalBytesSent_ = 128, totalAcksReceived_ = , totalSendMap_ = {[Key: Ok, Value: 4], }, totalLatencyAccumulator_ = Latencies [ 50pct: 4004.62ms, 90pct: 0ms, 99pct: 0ms, 99.9pct: 0ms]) +2017-06-20 18:04:54.310 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/1498006988testProducerTimeout, subscription-name, 0] , ConsumerStatsImpl (, numBytesRecieved_ = 0, totalNumBytesRecieved_ = 0, receivedMsgMap_ = {}, ackedMsgMap_ = {}, totalReceivedMsgMap_ = {}, totalAckedMsgMap_ = {}) +2017-06-20 18:04:54.310 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/1498006988testProducerTimeout, subscription-name, 1] , ConsumerStatsImpl (, numBytesRecieved_ = 0, totalNumBytesRecieved_ = 128, receivedMsgMap_ = {}, ackedMsgMap_ = {}, totalReceivedMsgMap_ = {[Key: Ok, Value: 4], }, totalAckedMsgMap_ = {[Key: {Result: Ok, ackType0}, Value: 4], }) +2017-06-20 18:04:55.297 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(81)- [persistent://property/cluster/namespace/1498006988testProducerTimeout, subscription-name, 0] ~ConsumerImpl +2017-06-20 18:04:55.297 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(81)- [persistent://property/cluster/namespace/1498006988testProducerTimeout, subscription-name, 1] ~ConsumerImpl +2017-06-20 18:04:55.297 : WARN /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(84)- [persistent://property/cluster/namespace/1498006988testProducerTimeout, subscription-name, 1] Destroyed consumer which was not properly closed +2017-06-20 18:04:55.297 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientImpl.cc(84)- ~ClientImpl +[ OK ] BatchMessageTest.testProducerTimeout (21035 ms) +[ RUN ] BatchMessageTest.testBatchSizeInBytes +2017-06-20 18:04:55.298 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConnectionPool.cc(62)- Created connection for pulsar://localhost:8885 +2017-06-20 18:04:55.300 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientConnection.cc(265)- [127.0.0.1:50151 -> 127.0.0.1:8885] Connected to broker +2017-06-20 18:04:55.301 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/BatchMessageContainer.cc(39)- { BatchContainer [size = 0] [batchSizeInBytes_ = 0] [maxAllowedMessageBatchSizeInBytes_ = 20] [maxAllowedNumMessagesInBatch_ = 1000] [topicName = persistent://property/cluster/namespace/1498006988testBatchSizeInBytes] [producerName_ = ] [batchSizeInBytes_ = 0] [numberOfBatchesSent = 0] [averageBatchSize = 0]} BatchMessageContainer constructed +2017-06-20 18:04:55.301 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/1498006988testBatchSizeInBytes, ] Getting connection from pool +2017-06-20 18:04:55.306 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ProducerImpl.cc(118)- [persistent://property/cluster/namespace/1498006988testBatchSizeInBytes, ] Created producer on broker [127.0.0.1:50151 -> 127.0.0.1:8885] +2017-06-20 18:04:55.307 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/1498006988testBatchSizeInBytes, subscription-name, 0] Getting connection from pool +2017-06-20 18:04:55.311 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(155)- [persistent://property/cluster/namespace/1498006988testBatchSizeInBytes, subscription-name, 0] Created consumer on broker [127.0.0.1:50151 -> 127.0.0.1:8885] +2017-06-20 18:04:55.312 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(639)- [persistent://property/cluster/namespace/1498006988testBatchSizeInBytes, subscription-name, 0] Closed consumer 0 +2017-06-20 18:04:55.312 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/1498006988testBatchSizeInBytes, subscription-name, 1] Getting connection from pool +2017-06-20 18:04:55.313 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(155)- [persistent://property/cluster/namespace/1498006988testBatchSizeInBytes, subscription-name, 1] Created consumer on broker [127.0.0.1:50151 -> 127.0.0.1:8885] +2017-06-20 18:05:00.329 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(81)- [persistent://property/cluster/namespace/1498006988testBatchSizeInBytes, subscription-name, 0] ~ConsumerImpl +2017-06-20 18:05:00.329 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(81)- [persistent://property/cluster/namespace/1498006988testBatchSizeInBytes, subscription-name, 1] ~ConsumerImpl +2017-06-20 18:05:00.329 : WARN /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(84)- [persistent://property/cluster/namespace/1498006988testBatchSizeInBytes, subscription-name, 1] Destroyed consumer which was not properly closed +2017-06-20 18:05:00.329 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientImpl.cc(84)- ~ClientImpl +[ OK ] BatchMessageTest.testBatchSizeInBytes (5033 ms) +[ RUN ] BatchMessageTest.testSmallReceiverQueueSize +2017-06-20 18:05:00.331 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConnectionPool.cc(62)- Created connection for pulsar://localhost:8885 +2017-06-20 18:05:00.332 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientConnection.cc(265)- [127.0.0.1:50153 -> 127.0.0.1:8885] Connected to broker +2017-06-20 18:05:00.333 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/BatchMessageContainer.cc(39)- { BatchContainer [size = 0] [batchSizeInBytes_ = 0] [maxAllowedMessageBatchSizeInBytes_ = 131072] [maxAllowedNumMessagesInBatch_ = 1000] [topicName = persistent://property/cluster/namespace/1498006988testSmallReceiverQueueSize] [producerName_ = ] [batchSizeInBytes_ = 0] [numberOfBatchesSent = 0] [averageBatchSize = 0]} BatchMessageContainer constructed +2017-06-20 18:05:00.333 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/1498006988testSmallReceiverQueueSize, ] Getting connection from pool +2017-06-20 18:05:00.338 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ProducerImpl.cc(118)- [persistent://property/cluster/namespace/1498006988testSmallReceiverQueueSize, ] Created producer on broker [127.0.0.1:50153 -> 127.0.0.1:8885] +2017-06-20 18:05:00.338 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/1498006988testSmallReceiverQueueSize, subscription-name, 0] Getting connection from pool +2017-06-20 18:05:00.342 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(155)- [persistent://property/cluster/namespace/1498006988testSmallReceiverQueueSize, subscription-name, 0] Created consumer on broker [127.0.0.1:50153 -> 127.0.0.1:8885] +2017-06-20 18:05:00.342 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(639)- [persistent://property/cluster/namespace/1498006988testSmallReceiverQueueSize, subscription-name, 0] Closed consumer 0 +2017-06-20 18:05:00.343 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/1498006988testSmallReceiverQueueSize, subscription-name, 1] Getting connection from pool +2017-06-20 18:05:00.343 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(155)- [persistent://property/cluster/namespace/1498006988testSmallReceiverQueueSize, subscription-name, 1] Created consumer on broker [127.0.0.1:50153 -> 127.0.0.1:8885] +2017-06-20 18:05:10.333 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ProducerStatsImpl.cc(90)- Producer [persistent://property/cluster/namespace/1498006988testSmallReceiverQueueSize, ] , ProducerStatsImpl (, numMsgsSent_ = 76751, numBytesSent_ = 3135681, sendMap_ = {[Key: Ok, Value: 76736], }, latencyAccumulator_ = Latencies [ 50pct: 2.96602ms, 90pct: 3.34363ms, 99pct: 4.6138ms, 99.9pct: 10.4885ms], totalMsgsSent_ = 76751, totalBytesSent_ = 3135681, totalAcksReceived_ = , totalSendMap_ = {[Key: Ok, Value: 76736], }, totalLatencyAccumulator_ = Latencies [ 50pct: 2.96602ms, 90pct: 3.34363ms, 99pct: 4.6138ms, 99.9pct: 10.4885ms]) +2017-06-20 18:05:10.338 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/1498006988testSmallReceiverQueueSize, subscription-name, 0] , ConsumerStatsImpl (, numBytesRecieved_ = 0, totalNumBytesRecieved_ = 0, receivedMsgMap_ = {}, ackedMsgMap_ = {}, totalReceivedMsgMap_ = {}, totalAckedMsgMap_ = {}) +2017-06-20 18:05:10.343 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/1498006988testSmallReceiverQueueSize, subscription-name, 1] , ConsumerStatsImpl (, numBytesRecieved_ = 0, totalNumBytesRecieved_ = 0, receivedMsgMap_ = {}, ackedMsgMap_ = {}, totalReceivedMsgMap_ = {}, totalAckedMsgMap_ = {}) +2017-06-20 18:05:20.337 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ProducerStatsImpl.cc(90)- Producer [persistent://property/cluster/namespace/1498006988testSmallReceiverQueueSize, ] , ProducerStatsImpl (, numMsgsSent_ = 23249, numBytesSent_ = 953209, sendMap_ = {[Key: Ok, Value: 23264], }, latencyAccumulator_ = Latencies [ 50pct: 2.96428ms, 90pct: 3.21813ms, 99pct: 4.46976ms, 99.9pct: 4.70051ms], totalMsgsSent_ = 100000, totalBytesSent_ = 4088890, totalAcksReceived_ = , totalSendMap_ = {[Key: Ok, Value: 100000], }, totalLatencyAccumulator_ = Latencies [ 50pct: 2.96553ms, 90pct: 3.29556ms, 99pct: 4.50829ms, 99.9pct: 9.72456ms]) +2017-06-20 18:05:20.339 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/1498006988testSmallReceiverQueueSize, subscription-name, 0] , ConsumerStatsImpl (, numBytesRecieved_ = 0, totalNumBytesRecieved_ = 0, receivedMsgMap_ = {}, ackedMsgMap_ = {}, totalReceivedMsgMap_ = {}, totalAckedMsgMap_ = {}) +2017-06-20 18:05:20.344 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/1498006988testSmallReceiverQueueSize, subscription-name, 1] , ConsumerStatsImpl (, numBytesRecieved_ = 4088890, totalNumBytesRecieved_ = 4088890, receivedMsgMap_ = {[Key: Ok, Value: 100000], [Key: TimeOut, Value: 1], }, ackedMsgMap_ = {[Key: {Result: Ok, ackType0}, Value: 100000], }, totalReceivedMsgMap_ = {[Key: Ok, Value: 100000], [Key: TimeOut, Value: 1], }, totalAckedMsgMap_ = {[Key: {Result: Ok, ackType0}, Value: 100000], }) +2017-06-20 18:05:25.859 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(81)- [persistent://property/cluster/namespace/1498006988testSmallReceiverQueueSize, subscription-name, 0] ~ConsumerImpl +2017-06-20 18:05:25.859 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(81)- [persistent://property/cluster/namespace/1498006988testSmallReceiverQueueSize, subscription-name, 1] ~ConsumerImpl +2017-06-20 18:05:25.859 : WARN /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(84)- [persistent://property/cluster/namespace/1498006988testSmallReceiverQueueSize, subscription-name, 1] Destroyed consumer which was not properly closed +2017-06-20 18:05:25.859 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientImpl.cc(84)- ~ClientImpl +[ OK ] BatchMessageTest.testSmallReceiverQueueSize (25531 ms) +[ RUN ] BatchMessageTest.testIndividualAck +2017-06-20 18:05:25.862 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConnectionPool.cc(62)- Created connection for pulsar://localhost:8885 +2017-06-20 18:05:25.863 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientConnection.cc(265)- [127.0.0.1:50162 -> 127.0.0.1:8885] Connected to broker +2017-06-20 18:05:25.864 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/BatchMessageContainer.cc(39)- { BatchContainer [size = 0] [batchSizeInBytes_ = 0] [maxAllowedMessageBatchSizeInBytes_ = 131072] [maxAllowedNumMessagesInBatch_ = 5] [topicName = persistent://property/cluster/namespace/1498006988testIndividualAck] [producerName_ = ] [batchSizeInBytes_ = 0] [numberOfBatchesSent = 0] [averageBatchSize = 0]} BatchMessageContainer constructed +2017-06-20 18:05:25.864 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/1498006988testIndividualAck, ] Getting connection from pool +2017-06-20 18:05:25.869 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ProducerImpl.cc(118)- [persistent://property/cluster/namespace/1498006988testIndividualAck, ] Created producer on broker [127.0.0.1:50162 -> 127.0.0.1:8885] +2017-06-20 18:05:25.869 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/1498006988testIndividualAck, subscription-name, 0] Getting connection from pool +2017-06-20 18:05:25.873 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(155)- [persistent://property/cluster/namespace/1498006988testIndividualAck, subscription-name, 0] Created consumer on broker [127.0.0.1:50162 -> 127.0.0.1:8885] +2017-06-20 18:05:25.873 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(639)- [persistent://property/cluster/namespace/1498006988testIndividualAck, subscription-name, 0] Closed consumer 0 +2017-06-20 18:05:25.874 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/1498006988testIndividualAck, subscription-name, 1] Getting connection from pool +2017-06-20 18:05:25.874 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(155)- [persistent://property/cluster/namespace/1498006988testIndividualAck, subscription-name, 1] Created consumer on broker [127.0.0.1:50162 -> 127.0.0.1:8885] +2017-06-20 18:05:26.869 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ProducerStatsImpl.cc(90)- Producer [persistent://property/cluster/namespace/1498006988testIndividualAck, ] , ProducerStatsImpl (, numMsgsSent_ = 10, numBytesSent_ = 280, sendMap_ = {[Key: Ok, Value: 10], }, latencyAccumulator_ = Latencies [ 50pct: 1.479ms, 90pct: 1.466ms, 99pct: 0.898ms, 99.9pct: 0.878ms], totalMsgsSent_ = 10, totalBytesSent_ = 280, totalAcksReceived_ = , totalSendMap_ = {[Key: Ok, Value: 10], }, totalLatencyAccumulator_ = Latencies [ 50pct: 1.479ms, 90pct: 1.466ms, 99pct: 0.898ms, 99.9pct: 0.878ms]) +2017-06-20 18:05:26.870 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/1498006988testIndividualAck, subscription-name, 0] , ConsumerStatsImpl (, numBytesRecieved_ = 0, totalNumBytesRecieved_ = 0, receivedMsgMap_ = {}, ackedMsgMap_ = {}, totalReceivedMsgMap_ = {}, totalAckedMsgMap_ = {}) +2017-06-20 18:05:26.875 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/1498006988testIndividualAck, subscription-name, 1] , ConsumerStatsImpl (, numBytesRecieved_ = 280, totalNumBytesRecieved_ = 280, receivedMsgMap_ = {[Key: Ok, Value: 10], }, ackedMsgMap_ = {[Key: {Result: Ok, ackType0}, Value: 5], }, totalReceivedMsgMap_ = {[Key: Ok, Value: 10], }, totalAckedMsgMap_ = {[Key: {Result: Ok, ackType0}, Value: 5], }) +2017-06-20 18:05:27.873 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ProducerStatsImpl.cc(90)- Producer [persistent://property/cluster/namespace/1498006988testIndividualAck, ] , ProducerStatsImpl (, numMsgsSent_ = 0, numBytesSent_ = 0, sendMap_ = {}, latencyAccumulator_ = Latencies [ 50pct: 0ms, 90pct: 0ms, 99pct: 0ms, 99.9pct: 0ms], totalMsgsSent_ = 10, totalBytesSent_ = 280, totalAcksReceived_ = , totalSendMap_ = {[Key: Ok, Value: 10], }, totalLatencyAccumulator_ = Latencies [ 50pct: 1.479ms, 90pct: 1.466ms, 99pct: 0.898ms, 99.9pct: 0.878ms]) +2017-06-20 18:05:27.873 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/1498006988testIndividualAck, subscription-name, 0] , ConsumerStatsImpl (, numBytesRecieved_ = 0, totalNumBytesRecieved_ = 0, receivedMsgMap_ = {}, ackedMsgMap_ = {}, totalReceivedMsgMap_ = {}, totalAckedMsgMap_ = {}) +2017-06-20 18:05:27.875 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/1498006988testIndividualAck, subscription-name, 1] , ConsumerStatsImpl (, numBytesRecieved_ = 0, totalNumBytesRecieved_ = 280, receivedMsgMap_ = {}, ackedMsgMap_ = {}, totalReceivedMsgMap_ = {[Key: Ok, Value: 10], }, totalAckedMsgMap_ = {[Key: {Result: Ok, ackType0}, Value: 5], }) +2017-06-20 18:05:28.877 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ProducerStatsImpl.cc(90)- Producer [persistent://property/cluster/namespace/1498006988testIndividualAck, ] , ProducerStatsImpl (, numMsgsSent_ = 0, numBytesSent_ = 0, sendMap_ = {}, latencyAccumulator_ = Latencies [ 50pct: 0ms, 90pct: 0ms, 99pct: 0ms, 99.9pct: 0ms], totalMsgsSent_ = 10, totalBytesSent_ = 280, totalAcksReceived_ = , totalSendMap_ = {[Key: Ok, Value: 10], }, totalLatencyAccumulator_ = Latencies [ 50pct: 1.479ms, 90pct: 1.466ms, 99pct: 0.898ms, 99.9pct: 0.878ms]) +2017-06-20 18:05:28.878 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/1498006988testIndividualAck, subscription-name, 0] , ConsumerStatsImpl (, numBytesRecieved_ = 0, totalNumBytesRecieved_ = 0, receivedMsgMap_ = {}, ackedMsgMap_ = {}, totalReceivedMsgMap_ = {}, totalAckedMsgMap_ = {}) +2017-06-20 18:05:28.878 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/1498006988testIndividualAck, subscription-name, 1] , ConsumerStatsImpl (, numBytesRecieved_ = 0, totalNumBytesRecieved_ = 280, receivedMsgMap_ = {}, ackedMsgMap_ = {}, totalReceivedMsgMap_ = {[Key: Ok, Value: 10], }, totalAckedMsgMap_ = {[Key: {Result: Ok, ackType0}, Value: 5], }) +2017-06-20 18:05:29.878 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ProducerStatsImpl.cc(90)- Producer [persistent://property/cluster/namespace/1498006988testIndividualAck, ] , ProducerStatsImpl (, numMsgsSent_ = 0, numBytesSent_ = 0, sendMap_ = {}, latencyAccumulator_ = Latencies [ 50pct: 0ms, 90pct: 0ms, 99pct: 0ms, 99.9pct: 0ms], totalMsgsSent_ = 10, totalBytesSent_ = 280, totalAcksReceived_ = , totalSendMap_ = {[Key: Ok, Value: 10], }, totalLatencyAccumulator_ = Latencies [ 50pct: 1.479ms, 90pct: 1.466ms, 99pct: 0.898ms, 99.9pct: 0.878ms]) +2017-06-20 18:05:29.878 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/1498006988testIndividualAck, subscription-name, 0] , ConsumerStatsImpl (, numBytesRecieved_ = 0, totalNumBytesRecieved_ = 0, receivedMsgMap_ = {}, ackedMsgMap_ = {}, totalReceivedMsgMap_ = {}, totalAckedMsgMap_ = {}) +2017-06-20 18:05:29.878 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/1498006988testIndividualAck, subscription-name, 1] , ConsumerStatsImpl (, numBytesRecieved_ = 0, totalNumBytesRecieved_ = 280, receivedMsgMap_ = {}, ackedMsgMap_ = {}, totalReceivedMsgMap_ = {[Key: Ok, Value: 10], }, totalAckedMsgMap_ = {[Key: {Result: Ok, ackType0}, Value: 5], }) +2017-06-20 18:05:30.880 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ProducerStatsImpl.cc(90)- Producer [persistent://property/cluster/namespace/1498006988testIndividualAck, ] , ProducerStatsImpl (, numMsgsSent_ = 0, numBytesSent_ = 0, sendMap_ = {}, latencyAccumulator_ = Latencies [ 50pct: 0ms, 90pct: 0ms, 99pct: 0ms, 99.9pct: 0ms], totalMsgsSent_ = 10, totalBytesSent_ = 280, totalAcksReceived_ = , totalSendMap_ = {[Key: Ok, Value: 10], }, totalLatencyAccumulator_ = Latencies [ 50pct: 1.479ms, 90pct: 1.466ms, 99pct: 0.898ms, 99.9pct: 0.878ms]) +2017-06-20 18:05:30.880 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/1498006988testIndividualAck, subscription-name, 0] , ConsumerStatsImpl (, numBytesRecieved_ = 0, totalNumBytesRecieved_ = 0, receivedMsgMap_ = {}, ackedMsgMap_ = {}, totalReceivedMsgMap_ = {}, totalAckedMsgMap_ = {}) +2017-06-20 18:05:30.880 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/1498006988testIndividualAck, subscription-name, 1] , ConsumerStatsImpl (, numBytesRecieved_ = 0, totalNumBytesRecieved_ = 280, receivedMsgMap_ = {}, ackedMsgMap_ = {}, totalReceivedMsgMap_ = {[Key: Ok, Value: 10], }, totalAckedMsgMap_ = {[Key: {Result: Ok, ackType0}, Value: 5], }) +2017-06-20 18:05:30.888 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(639)- [persistent://property/cluster/namespace/1498006988testIndividualAck, subscription-name, 1] Closed consumer 1 +2017-06-20 18:05:30.889 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/1498006988testIndividualAck, subscription-name, 2] Getting connection from pool +2017-06-20 18:05:30.890 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(155)- [persistent://property/cluster/namespace/1498006988testIndividualAck, subscription-name, 2] Created consumer on broker [127.0.0.1:50162 -> 127.0.0.1:8885] +2017-06-20 18:05:30.890 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(81)- [persistent://property/cluster/namespace/1498006988testIndividualAck, subscription-name, 1] ~ConsumerImpl +2017-06-20 18:05:31.884 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ProducerStatsImpl.cc(90)- Producer [persistent://property/cluster/namespace/1498006988testIndividualAck, ] , ProducerStatsImpl (, numMsgsSent_ = 0, numBytesSent_ = 0, sendMap_ = {}, latencyAccumulator_ = Latencies [ 50pct: 0ms, 90pct: 0ms, 99pct: 0ms, 99.9pct: 0ms], totalMsgsSent_ = 10, totalBytesSent_ = 280, totalAcksReceived_ = , totalSendMap_ = {[Key: Ok, Value: 10], }, totalLatencyAccumulator_ = Latencies [ 50pct: 1.479ms, 90pct: 1.466ms, 99pct: 0.898ms, 99.9pct: 0.878ms]) +2017-06-20 18:05:31.884 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/1498006988testIndividualAck, subscription-name, 0] , ConsumerStatsImpl (, numBytesRecieved_ = 0, totalNumBytesRecieved_ = 0, receivedMsgMap_ = {}, ackedMsgMap_ = {}, totalReceivedMsgMap_ = {}, totalAckedMsgMap_ = {}) +2017-06-20 18:05:31.890 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/1498006988testIndividualAck, subscription-name, 2] , ConsumerStatsImpl (, numBytesRecieved_ = 280, totalNumBytesRecieved_ = 280, receivedMsgMap_ = {[Key: Ok, Value: 10], }, ackedMsgMap_ = {[Key: {Result: Ok, ackType0}, Value: 6], }, totalReceivedMsgMap_ = {[Key: Ok, Value: 10], }, totalAckedMsgMap_ = {[Key: {Result: Ok, ackType0}, Value: 6], }) +2017-06-20 18:05:32.889 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ProducerStatsImpl.cc(90)- Producer [persistent://property/cluster/namespace/1498006988testIndividualAck, ] , ProducerStatsImpl (, numMsgsSent_ = 0, numBytesSent_ = 0, sendMap_ = {}, latencyAccumulator_ = Latencies [ 50pct: 0ms, 90pct: 0ms, 99pct: 0ms, 99.9pct: 0ms], totalMsgsSent_ = 10, totalBytesSent_ = 280, totalAcksReceived_ = , totalSendMap_ = {[Key: Ok, Value: 10], }, totalLatencyAccumulator_ = Latencies [ 50pct: 1.479ms, 90pct: 1.466ms, 99pct: 0.898ms, 99.9pct: 0.878ms]) +2017-06-20 18:05:32.889 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/1498006988testIndividualAck, subscription-name, 0] , ConsumerStatsImpl (, numBytesRecieved_ = 0, totalNumBytesRecieved_ = 0, receivedMsgMap_ = {}, ackedMsgMap_ = {}, totalReceivedMsgMap_ = {}, totalAckedMsgMap_ = {}) +2017-06-20 18:05:32.890 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/1498006988testIndividualAck, subscription-name, 2] , ConsumerStatsImpl (, numBytesRecieved_ = 0, totalNumBytesRecieved_ = 280, receivedMsgMap_ = {}, ackedMsgMap_ = {}, totalReceivedMsgMap_ = {[Key: Ok, Value: 10], }, totalAckedMsgMap_ = {[Key: {Result: Ok, ackType0}, Value: 6], }) +2017-06-20 18:05:33.893 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ProducerStatsImpl.cc(90)- Producer [persistent://property/cluster/namespace/1498006988testIndividualAck, ] , ProducerStatsImpl (, numMsgsSent_ = 0, numBytesSent_ = 0, sendMap_ = {}, latencyAccumulator_ = Latencies [ 50pct: 0ms, 90pct: 0ms, 99pct: 0ms, 99.9pct: 0ms], totalMsgsSent_ = 10, totalBytesSent_ = 280, totalAcksReceived_ = , totalSendMap_ = {[Key: Ok, Value: 10], }, totalLatencyAccumulator_ = Latencies [ 50pct: 1.479ms, 90pct: 1.466ms, 99pct: 0.898ms, 99.9pct: 0.878ms]) +2017-06-20 18:05:33.893 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/1498006988testIndividualAck, subscription-name, 0] , ConsumerStatsImpl (, numBytesRecieved_ = 0, totalNumBytesRecieved_ = 0, receivedMsgMap_ = {}, ackedMsgMap_ = {}, totalReceivedMsgMap_ = {}, totalAckedMsgMap_ = {}) +2017-06-20 18:05:33.893 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/1498006988testIndividualAck, subscription-name, 2] , ConsumerStatsImpl (, numBytesRecieved_ = 0, totalNumBytesRecieved_ = 280, receivedMsgMap_ = {}, ackedMsgMap_ = {}, totalReceivedMsgMap_ = {[Key: Ok, Value: 10], }, totalAckedMsgMap_ = {[Key: {Result: Ok, ackType0}, Value: 6], }) +2017-06-20 18:05:34.897 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ProducerStatsImpl.cc(90)- Producer [persistent://property/cluster/namespace/1498006988testIndividualAck, ] , ProducerStatsImpl (, numMsgsSent_ = 0, numBytesSent_ = 0, sendMap_ = {}, latencyAccumulator_ = Latencies [ 50pct: 0ms, 90pct: 0ms, 99pct: 0ms, 99.9pct: 0ms], totalMsgsSent_ = 10, totalBytesSent_ = 280, totalAcksReceived_ = , totalSendMap_ = {[Key: Ok, Value: 10], }, totalLatencyAccumulator_ = Latencies [ 50pct: 1.479ms, 90pct: 1.466ms, 99pct: 0.898ms, 99.9pct: 0.878ms]) +2017-06-20 18:05:34.897 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/1498006988testIndividualAck, subscription-name, 0] , ConsumerStatsImpl (, numBytesRecieved_ = 0, totalNumBytesRecieved_ = 0, receivedMsgMap_ = {}, ackedMsgMap_ = {}, totalReceivedMsgMap_ = {}, totalAckedMsgMap_ = {}) +2017-06-20 18:05:34.897 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/1498006988testIndividualAck, subscription-name, 2] , ConsumerStatsImpl (, numBytesRecieved_ = 0, totalNumBytesRecieved_ = 280, receivedMsgMap_ = {}, ackedMsgMap_ = {}, totalReceivedMsgMap_ = {[Key: Ok, Value: 10], }, totalAckedMsgMap_ = {[Key: {Result: Ok, ackType0}, Value: 6], }) +2017-06-20 18:05:35.896 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(639)- [persistent://property/cluster/namespace/1498006988testIndividualAck, subscription-name, 2] Closed consumer 2 +2017-06-20 18:05:35.897 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/1498006988testIndividualAck, subscription-name, 3] Getting connection from pool +2017-06-20 18:05:35.897 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ProducerStatsImpl.cc(90)- Producer [persistent://property/cluster/namespace/1498006988testIndividualAck, ] , ProducerStatsImpl (, numMsgsSent_ = 0, numBytesSent_ = 0, sendMap_ = {}, latencyAccumulator_ = Latencies [ 50pct: 0ms, 90pct: 0ms, 99pct: 0ms, 99.9pct: 0ms], totalMsgsSent_ = 10, totalBytesSent_ = 280, totalAcksReceived_ = , totalSendMap_ = {[Key: Ok, Value: 10], }, totalLatencyAccumulator_ = Latencies [ 50pct: 1.479ms, 90pct: 1.466ms, 99pct: 0.898ms, 99.9pct: 0.878ms]) +2017-06-20 18:05:35.897 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/1498006988testIndividualAck, subscription-name, 0] , ConsumerStatsImpl (, numBytesRecieved_ = 0, totalNumBytesRecieved_ = 0, receivedMsgMap_ = {}, ackedMsgMap_ = {}, totalReceivedMsgMap_ = {}, totalAckedMsgMap_ = {}) +2017-06-20 18:05:35.897 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/1498006988testIndividualAck, subscription-name, 2] , ConsumerStatsImpl (, numBytesRecieved_ = 0, totalNumBytesRecieved_ = 280, receivedMsgMap_ = {[Key: TimeOut, Value: 1], }, ackedMsgMap_ = {}, totalReceivedMsgMap_ = {[Key: Ok, Value: 10], [Key: TimeOut, Value: 1], }, totalAckedMsgMap_ = {[Key: {Result: Ok, ackType0}, Value: 6], }) +2017-06-20 18:05:35.898 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(155)- [persistent://property/cluster/namespace/1498006988testIndividualAck, subscription-name, 3] Created consumer on broker [127.0.0.1:50162 -> 127.0.0.1:8885] +2017-06-20 18:05:35.898 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(81)- [persistent://property/cluster/namespace/1498006988testIndividualAck, subscription-name, 2] ~ConsumerImpl +2017-06-20 18:05:36.899 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/1498006988testIndividualAck, subscription-name, 3] , ConsumerStatsImpl (, numBytesRecieved_ = 140, totalNumBytesRecieved_ = 140, receivedMsgMap_ = {[Key: Ok, Value: 5], }, ackedMsgMap_ = {[Key: {Result: Ok, ackType0}, Value: 4], }, totalReceivedMsgMap_ = {[Key: Ok, Value: 5], }, totalAckedMsgMap_ = {[Key: {Result: Ok, ackType0}, Value: 4], }) +2017-06-20 18:05:36.900 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ProducerStatsImpl.cc(90)- Producer [persistent://property/cluster/namespace/1498006988testIndividualAck, ] , ProducerStatsImpl (, numMsgsSent_ = 0, numBytesSent_ = 0, sendMap_ = {}, latencyAccumulator_ = Latencies [ 50pct: 0ms, 90pct: 0ms, 99pct: 0ms, 99.9pct: 0ms], totalMsgsSent_ = 10, totalBytesSent_ = 280, totalAcksReceived_ = , totalSendMap_ = {[Key: Ok, Value: 10], }, totalLatencyAccumulator_ = Latencies [ 50pct: 1.479ms, 90pct: 1.466ms, 99pct: 0.898ms, 99.9pct: 0.878ms]) +2017-06-20 18:05:36.900 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/1498006988testIndividualAck, subscription-name, 0] , ConsumerStatsImpl (, numBytesRecieved_ = 0, totalNumBytesRecieved_ = 0, receivedMsgMap_ = {}, ackedMsgMap_ = {}, totalReceivedMsgMap_ = {}, totalAckedMsgMap_ = {}) +2017-06-20 18:05:37.903 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/1498006988testIndividualAck, subscription-name, 3] , ConsumerStatsImpl (, numBytesRecieved_ = 0, totalNumBytesRecieved_ = 140, receivedMsgMap_ = {}, ackedMsgMap_ = {}, totalReceivedMsgMap_ = {[Key: Ok, Value: 5], }, totalAckedMsgMap_ = {[Key: {Result: Ok, ackType0}, Value: 4], }) +2017-06-20 18:05:37.903 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ProducerStatsImpl.cc(90)- Producer [persistent://property/cluster/namespace/1498006988testIndividualAck, ] , ProducerStatsImpl (, numMsgsSent_ = 0, numBytesSent_ = 0, sendMap_ = {}, latencyAccumulator_ = Latencies [ 50pct: 0ms, 90pct: 0ms, 99pct: 0ms, 99.9pct: 0ms], totalMsgsSent_ = 10, totalBytesSent_ = 280, totalAcksReceived_ = , totalSendMap_ = {[Key: Ok, Value: 10], }, totalLatencyAccumulator_ = Latencies [ 50pct: 1.479ms, 90pct: 1.466ms, 99pct: 0.898ms, 99.9pct: 0.878ms]) +2017-06-20 18:05:37.903 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/1498006988testIndividualAck, subscription-name, 0] , ConsumerStatsImpl (, numBytesRecieved_ = 0, totalNumBytesRecieved_ = 0, receivedMsgMap_ = {}, ackedMsgMap_ = {}, totalReceivedMsgMap_ = {}, totalAckedMsgMap_ = {}) +2017-06-20 18:05:38.907 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/1498006988testIndividualAck, subscription-name, 3] , ConsumerStatsImpl (, numBytesRecieved_ = 0, totalNumBytesRecieved_ = 140, receivedMsgMap_ = {}, ackedMsgMap_ = {}, totalReceivedMsgMap_ = {[Key: Ok, Value: 5], }, totalAckedMsgMap_ = {[Key: {Result: Ok, ackType0}, Value: 4], }) +2017-06-20 18:05:38.907 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ProducerStatsImpl.cc(90)- Producer [persistent://property/cluster/namespace/1498006988testIndividualAck, ] , ProducerStatsImpl (, numMsgsSent_ = 0, numBytesSent_ = 0, sendMap_ = {}, latencyAccumulator_ = Latencies [ 50pct: 0ms, 90pct: 0ms, 99pct: 0ms, 99.9pct: 0ms], totalMsgsSent_ = 10, totalBytesSent_ = 280, totalAcksReceived_ = , totalSendMap_ = {[Key: Ok, Value: 10], }, totalLatencyAccumulator_ = Latencies [ 50pct: 1.479ms, 90pct: 1.466ms, 99pct: 0.898ms, 99.9pct: 0.878ms]) +2017-06-20 18:05:38.907 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/1498006988testIndividualAck, subscription-name, 0] , ConsumerStatsImpl (, numBytesRecieved_ = 0, totalNumBytesRecieved_ = 0, receivedMsgMap_ = {}, ackedMsgMap_ = {}, totalReceivedMsgMap_ = {}, totalAckedMsgMap_ = {}) +2017-06-20 18:05:39.907 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/1498006988testIndividualAck, subscription-name, 3] , ConsumerStatsImpl (, numBytesRecieved_ = 0, totalNumBytesRecieved_ = 140, receivedMsgMap_ = {}, ackedMsgMap_ = {}, totalReceivedMsgMap_ = {[Key: Ok, Value: 5], }, totalAckedMsgMap_ = {[Key: {Result: Ok, ackType0}, Value: 4], }) +2017-06-20 18:05:39.907 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ProducerStatsImpl.cc(90)- Producer [persistent://property/cluster/namespace/1498006988testIndividualAck, ] , ProducerStatsImpl (, numMsgsSent_ = 0, numBytesSent_ = 0, sendMap_ = {}, latencyAccumulator_ = Latencies [ 50pct: 0ms, 90pct: 0ms, 99pct: 0ms, 99.9pct: 0ms], totalMsgsSent_ = 10, totalBytesSent_ = 280, totalAcksReceived_ = , totalSendMap_ = {[Key: Ok, Value: 10], }, totalLatencyAccumulator_ = Latencies [ 50pct: 1.479ms, 90pct: 1.466ms, 99pct: 0.898ms, 99.9pct: 0.878ms]) +2017-06-20 18:05:39.907 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/1498006988testIndividualAck, subscription-name, 0] , ConsumerStatsImpl (, numBytesRecieved_ = 0, totalNumBytesRecieved_ = 0, receivedMsgMap_ = {}, ackedMsgMap_ = {}, totalReceivedMsgMap_ = {}, totalAckedMsgMap_ = {}) +2017-06-20 18:05:40.900 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(639)- [persistent://property/cluster/namespace/1498006988testIndividualAck, subscription-name, 3] Closed consumer 3 +2017-06-20 18:05:40.901 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/1498006988testIndividualAck, subscription-name, 4] Getting connection from pool +2017-06-20 18:05:40.902 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(155)- [persistent://property/cluster/namespace/1498006988testIndividualAck, subscription-name, 4] Created consumer on broker [127.0.0.1:50162 -> 127.0.0.1:8885] +2017-06-20 18:05:40.902 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(81)- [persistent://property/cluster/namespace/1498006988testIndividualAck, subscription-name, 3] ~ConsumerImpl +2017-06-20 18:05:40.909 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ProducerStatsImpl.cc(90)- Producer [persistent://property/cluster/namespace/1498006988testIndividualAck, ] , ProducerStatsImpl (, numMsgsSent_ = 0, numBytesSent_ = 0, sendMap_ = {}, latencyAccumulator_ = Latencies [ 50pct: 0ms, 90pct: 0ms, 99pct: 0ms, 99.9pct: 0ms], totalMsgsSent_ = 10, totalBytesSent_ = 280, totalAcksReceived_ = , totalSendMap_ = {[Key: Ok, Value: 10], }, totalLatencyAccumulator_ = Latencies [ 50pct: 1.479ms, 90pct: 1.466ms, 99pct: 0.898ms, 99.9pct: 0.878ms]) +2017-06-20 18:05:40.909 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/1498006988testIndividualAck, subscription-name, 0] , ConsumerStatsImpl (, numBytesRecieved_ = 0, totalNumBytesRecieved_ = 0, receivedMsgMap_ = {}, ackedMsgMap_ = {}, totalReceivedMsgMap_ = {}, totalAckedMsgMap_ = {}) +2017-06-20 18:05:41.906 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/1498006988testIndividualAck, subscription-name, 4] , ConsumerStatsImpl (, numBytesRecieved_ = 140, totalNumBytesRecieved_ = 140, receivedMsgMap_ = {[Key: Ok, Value: 5], }, ackedMsgMap_ = {[Key: {Result: Ok, ackType0}, Value: 5], }, totalReceivedMsgMap_ = {[Key: Ok, Value: 5], }, totalAckedMsgMap_ = {[Key: {Result: Ok, ackType0}, Value: 5], }) +2017-06-20 18:05:41.909 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ProducerStatsImpl.cc(90)- Producer [persistent://property/cluster/namespace/1498006988testIndividualAck, ] , ProducerStatsImpl (, numMsgsSent_ = 0, numBytesSent_ = 0, sendMap_ = {}, latencyAccumulator_ = Latencies [ 50pct: 0ms, 90pct: 0ms, 99pct: 0ms, 99.9pct: 0ms], totalMsgsSent_ = 10, totalBytesSent_ = 280, totalAcksReceived_ = , totalSendMap_ = {[Key: Ok, Value: 10], }, totalLatencyAccumulator_ = Latencies [ 50pct: 1.479ms, 90pct: 1.466ms, 99pct: 0.898ms, 99.9pct: 0.878ms]) +2017-06-20 18:05:41.909 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/1498006988testIndividualAck, subscription-name, 0] , ConsumerStatsImpl (, numBytesRecieved_ = 0, totalNumBytesRecieved_ = 0, receivedMsgMap_ = {}, ackedMsgMap_ = {}, totalReceivedMsgMap_ = {}, totalAckedMsgMap_ = {}) +2017-06-20 18:05:42.907 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/1498006988testIndividualAck, subscription-name, 4] , ConsumerStatsImpl (, numBytesRecieved_ = 0, totalNumBytesRecieved_ = 140, receivedMsgMap_ = {}, ackedMsgMap_ = {}, totalReceivedMsgMap_ = {[Key: Ok, Value: 5], }, totalAckedMsgMap_ = {[Key: {Result: Ok, ackType0}, Value: 5], }) +2017-06-20 18:05:42.909 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ProducerStatsImpl.cc(90)- Producer [persistent://property/cluster/namespace/1498006988testIndividualAck, ] , ProducerStatsImpl (, numMsgsSent_ = 0, numBytesSent_ = 0, sendMap_ = {}, latencyAccumulator_ = Latencies [ 50pct: 0ms, 90pct: 0ms, 99pct: 0ms, 99.9pct: 0ms], totalMsgsSent_ = 10, totalBytesSent_ = 280, totalAcksReceived_ = , totalSendMap_ = {[Key: Ok, Value: 10], }, totalLatencyAccumulator_ = Latencies [ 50pct: 1.479ms, 90pct: 1.466ms, 99pct: 0.898ms, 99.9pct: 0.878ms]) +2017-06-20 18:05:42.909 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/1498006988testIndividualAck, subscription-name, 0] , ConsumerStatsImpl (, numBytesRecieved_ = 0, totalNumBytesRecieved_ = 0, receivedMsgMap_ = {}, ackedMsgMap_ = {}, totalReceivedMsgMap_ = {}, totalAckedMsgMap_ = {}) +2017-06-20 18:05:43.911 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/1498006988testIndividualAck, subscription-name, 4] , ConsumerStatsImpl (, numBytesRecieved_ = 0, totalNumBytesRecieved_ = 140, receivedMsgMap_ = {}, ackedMsgMap_ = {}, totalReceivedMsgMap_ = {[Key: Ok, Value: 5], }, totalAckedMsgMap_ = {[Key: {Result: Ok, ackType0}, Value: 5], }) +2017-06-20 18:05:43.911 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ProducerStatsImpl.cc(90)- Producer [persistent://property/cluster/namespace/1498006988testIndividualAck, ] , ProducerStatsImpl (, numMsgsSent_ = 0, numBytesSent_ = 0, sendMap_ = {}, latencyAccumulator_ = Latencies [ 50pct: 0ms, 90pct: 0ms, 99pct: 0ms, 99.9pct: 0ms], totalMsgsSent_ = 10, totalBytesSent_ = 280, totalAcksReceived_ = , totalSendMap_ = {[Key: Ok, Value: 10], }, totalLatencyAccumulator_ = Latencies [ 50pct: 1.479ms, 90pct: 1.466ms, 99pct: 0.898ms, 99.9pct: 0.878ms]) +2017-06-20 18:05:43.911 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/1498006988testIndividualAck, subscription-name, 0] , ConsumerStatsImpl (, numBytesRecieved_ = 0, totalNumBytesRecieved_ = 0, receivedMsgMap_ = {}, ackedMsgMap_ = {}, totalReceivedMsgMap_ = {}, totalAckedMsgMap_ = {}) +2017-06-20 18:05:44.913 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/1498006988testIndividualAck, subscription-name, 4] , ConsumerStatsImpl (, numBytesRecieved_ = 0, totalNumBytesRecieved_ = 140, receivedMsgMap_ = {}, ackedMsgMap_ = {}, totalReceivedMsgMap_ = {[Key: Ok, Value: 5], }, totalAckedMsgMap_ = {[Key: {Result: Ok, ackType0}, Value: 5], }) +2017-06-20 18:05:44.913 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ProducerStatsImpl.cc(90)- Producer [persistent://property/cluster/namespace/1498006988testIndividualAck, ] , ProducerStatsImpl (, numMsgsSent_ = 0, numBytesSent_ = 0, sendMap_ = {}, latencyAccumulator_ = Latencies [ 50pct: 0ms, 90pct: 0ms, 99pct: 0ms, 99.9pct: 0ms], totalMsgsSent_ = 10, totalBytesSent_ = 280, totalAcksReceived_ = , totalSendMap_ = {[Key: Ok, Value: 10], }, totalLatencyAccumulator_ = Latencies [ 50pct: 1.479ms, 90pct: 1.466ms, 99pct: 0.898ms, 99.9pct: 0.878ms]) +2017-06-20 18:05:44.913 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/1498006988testIndividualAck, subscription-name, 0] , ConsumerStatsImpl (, numBytesRecieved_ = 0, totalNumBytesRecieved_ = 0, receivedMsgMap_ = {}, ackedMsgMap_ = {}, totalReceivedMsgMap_ = {}, totalAckedMsgMap_ = {}) +2017-06-20 18:05:45.904 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(639)- [persistent://property/cluster/namespace/1498006988testIndividualAck, subscription-name, 4] Closed consumer 4 +2017-06-20 18:05:45.904 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/1498006988testIndividualAck, subscription-name, 5] Getting connection from pool +2017-06-20 18:05:45.905 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(155)- [persistent://property/cluster/namespace/1498006988testIndividualAck, subscription-name, 5] Created consumer on broker [127.0.0.1:50162 -> 127.0.0.1:8885] +2017-06-20 18:05:45.905 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(81)- [persistent://property/cluster/namespace/1498006988testIndividualAck, subscription-name, 4] ~ConsumerImpl +2017-06-20 18:05:45.915 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ProducerStatsImpl.cc(90)- Producer [persistent://property/cluster/namespace/1498006988testIndividualAck, ] , ProducerStatsImpl (, numMsgsSent_ = 0, numBytesSent_ = 0, sendMap_ = {}, latencyAccumulator_ = Latencies [ 50pct: 0ms, 90pct: 0ms, 99pct: 0ms, 99.9pct: 0ms], totalMsgsSent_ = 10, totalBytesSent_ = 280, totalAcksReceived_ = , totalSendMap_ = {[Key: Ok, Value: 10], }, totalLatencyAccumulator_ = Latencies [ 50pct: 1.479ms, 90pct: 1.466ms, 99pct: 0.898ms, 99.9pct: 0.878ms]) +2017-06-20 18:05:45.915 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/1498006988testIndividualAck, subscription-name, 0] , ConsumerStatsImpl (, numBytesRecieved_ = 0, totalNumBytesRecieved_ = 0, receivedMsgMap_ = {}, ackedMsgMap_ = {}, totalReceivedMsgMap_ = {}, totalAckedMsgMap_ = {}) +2017-06-20 18:05:46.907 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/1498006988testIndividualAck, subscription-name, 5] , ConsumerStatsImpl (, numBytesRecieved_ = 0, totalNumBytesRecieved_ = 0, receivedMsgMap_ = {}, ackedMsgMap_ = {}, totalReceivedMsgMap_ = {}, totalAckedMsgMap_ = {}) +2017-06-20 18:05:46.916 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ProducerStatsImpl.cc(90)- Producer [persistent://property/cluster/namespace/1498006988testIndividualAck, ] , ProducerStatsImpl (, numMsgsSent_ = 0, numBytesSent_ = 0, sendMap_ = {}, latencyAccumulator_ = Latencies [ 50pct: 0ms, 90pct: 0ms, 99pct: 0ms, 99.9pct: 0ms], totalMsgsSent_ = 10, totalBytesSent_ = 280, totalAcksReceived_ = , totalSendMap_ = {[Key: Ok, Value: 10], }, totalLatencyAccumulator_ = Latencies [ 50pct: 1.479ms, 90pct: 1.466ms, 99pct: 0.898ms, 99.9pct: 0.878ms]) +2017-06-20 18:05:46.916 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/1498006988testIndividualAck, subscription-name, 0] , ConsumerStatsImpl (, numBytesRecieved_ = 0, totalNumBytesRecieved_ = 0, receivedMsgMap_ = {}, ackedMsgMap_ = {}, totalReceivedMsgMap_ = {}, totalAckedMsgMap_ = {}) +2017-06-20 18:05:47.911 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/1498006988testIndividualAck, subscription-name, 5] , ConsumerStatsImpl (, numBytesRecieved_ = 0, totalNumBytesRecieved_ = 0, receivedMsgMap_ = {}, ackedMsgMap_ = {}, totalReceivedMsgMap_ = {}, totalAckedMsgMap_ = {}) +2017-06-20 18:05:47.917 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ProducerStatsImpl.cc(90)- Producer [persistent://property/cluster/namespace/1498006988testIndividualAck, ] , ProducerStatsImpl (, numMsgsSent_ = 0, numBytesSent_ = 0, sendMap_ = {}, latencyAccumulator_ = Latencies [ 50pct: 0ms, 90pct: 0ms, 99pct: 0ms, 99.9pct: 0ms], totalMsgsSent_ = 10, totalBytesSent_ = 280, totalAcksReceived_ = , totalSendMap_ = {[Key: Ok, Value: 10], }, totalLatencyAccumulator_ = Latencies [ 50pct: 1.479ms, 90pct: 1.466ms, 99pct: 0.898ms, 99.9pct: 0.878ms]) +2017-06-20 18:05:47.917 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/1498006988testIndividualAck, subscription-name, 0] , ConsumerStatsImpl (, numBytesRecieved_ = 0, totalNumBytesRecieved_ = 0, receivedMsgMap_ = {}, ackedMsgMap_ = {}, totalReceivedMsgMap_ = {}, totalAckedMsgMap_ = {}) +2017-06-20 18:05:48.916 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/1498006988testIndividualAck, subscription-name, 5] , ConsumerStatsImpl (, numBytesRecieved_ = 0, totalNumBytesRecieved_ = 0, receivedMsgMap_ = {}, ackedMsgMap_ = {}, totalReceivedMsgMap_ = {}, totalAckedMsgMap_ = {}) +2017-06-20 18:05:48.917 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ProducerStatsImpl.cc(90)- Producer [persistent://property/cluster/namespace/1498006988testIndividualAck, ] , ProducerStatsImpl (, numMsgsSent_ = 0, numBytesSent_ = 0, sendMap_ = {}, latencyAccumulator_ = Latencies [ 50pct: 0ms, 90pct: 0ms, 99pct: 0ms, 99.9pct: 0ms], totalMsgsSent_ = 10, totalBytesSent_ = 280, totalAcksReceived_ = , totalSendMap_ = {[Key: Ok, Value: 10], }, totalLatencyAccumulator_ = Latencies [ 50pct: 1.479ms, 90pct: 1.466ms, 99pct: 0.898ms, 99.9pct: 0.878ms]) +2017-06-20 18:05:48.917 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/1498006988testIndividualAck, subscription-name, 0] , ConsumerStatsImpl (, numBytesRecieved_ = 0, totalNumBytesRecieved_ = 0, receivedMsgMap_ = {}, ackedMsgMap_ = {}, totalReceivedMsgMap_ = {}, totalAckedMsgMap_ = {}) +2017-06-20 18:05:49.916 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/1498006988testIndividualAck, subscription-name, 5] , ConsumerStatsImpl (, numBytesRecieved_ = 0, totalNumBytesRecieved_ = 0, receivedMsgMap_ = {}, ackedMsgMap_ = {}, totalReceivedMsgMap_ = {}, totalAckedMsgMap_ = {}) +2017-06-20 18:05:49.918 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ProducerStatsImpl.cc(90)- Producer [persistent://property/cluster/namespace/1498006988testIndividualAck, ] , ProducerStatsImpl (, numMsgsSent_ = 0, numBytesSent_ = 0, sendMap_ = {}, latencyAccumulator_ = Latencies [ 50pct: 0ms, 90pct: 0ms, 99pct: 0ms, 99.9pct: 0ms], totalMsgsSent_ = 10, totalBytesSent_ = 280, totalAcksReceived_ = , totalSendMap_ = {[Key: Ok, Value: 10], }, totalLatencyAccumulator_ = Latencies [ 50pct: 1.479ms, 90pct: 1.466ms, 99pct: 0.898ms, 99.9pct: 0.878ms]) +2017-06-20 18:05:49.918 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/stats/ConsumerStatsImpl.cc(64)- Consumer [persistent://property/cluster/namespace/1498006988testIndividualAck, subscription-name, 0] , ConsumerStatsImpl (, numBytesRecieved_ = 0, totalNumBytesRecieved_ = 0, receivedMsgMap_ = {}, ackedMsgMap_ = {}, totalReceivedMsgMap_ = {}, totalAckedMsgMap_ = {}) +2017-06-20 18:05:50.906 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(81)- [persistent://property/cluster/namespace/1498006988testIndividualAck, subscription-name, 0] ~ConsumerImpl +2017-06-20 18:05:50.906 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(81)- [persistent://property/cluster/namespace/1498006988testIndividualAck, subscription-name, 5] ~ConsumerImpl +2017-06-20 18:05:50.906 : WARN /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(84)- [persistent://property/cluster/namespace/1498006988testIndividualAck, subscription-name, 5] Destroyed consumer which was not properly closed +2017-06-20 18:05:50.907 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientImpl.cc(84)- ~ClientImpl +[ OK ] BatchMessageTest.testIndividualAck (25046 ms) +[ RUN ] BatchMessageTest.testCumulativeAck +2017-06-20 18:05:50.908 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConnectionPool.cc(62)- Created connection for pulsar://localhost:8885 +2017-06-20 18:05:50.909 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientConnection.cc(265)- [127.0.0.1:50166 -> 127.0.0.1:8885] Connected to broker +2017-06-20 18:05:50.910 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/BatchMessageContainer.cc(39)- { BatchContainer [size = 0] [batchSizeInBytes_ = 0] [maxAllowedMessageBatchSizeInBytes_ = 131072] [maxAllowedNumMessagesInBatch_ = 5] [topicName = persistent://property/cluster/namespace/1498006988testCumulativeAck] [producerName_ = ] [batchSizeInBytes_ = 0] [numberOfBatchesSent = 0] [averageBatchSize = 0]} BatchMessageContainer constructed +2017-06-20 18:05:50.910 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/1498006988testCumulativeAck, ] Getting connection from pool +2017-06-20 18:05:50.916 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ProducerImpl.cc(118)- [persistent://property/cluster/namespace/1498006988testCumulativeAck, ] Created producer on broker [127.0.0.1:50166 -> 127.0.0.1:8885] +2017-06-20 18:05:50.916 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/1498006988testCumulativeAck, subscription-name, 0] Getting connection from pool +2017-06-20 18:05:50.920 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(155)- [persistent://property/cluster/namespace/1498006988testCumulativeAck, subscription-name, 0] Created consumer on broker [127.0.0.1:50166 -> 127.0.0.1:8885] +2017-06-20 18:05:50.920 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(639)- [persistent://property/cluster/namespace/1498006988testCumulativeAck, subscription-name, 0] Closed consumer 0 +2017-06-20 18:05:50.921 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/1498006988testCumulativeAck, subscription-name, 1] Getting connection from pool +2017-06-20 18:05:50.921 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(155)- [persistent://property/cluster/namespace/1498006988testCumulativeAck, subscription-name, 1] Created consumer on broker [127.0.0.1:50166 -> 127.0.0.1:8885] +2017-06-20 18:05:55.937 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(639)- [persistent://property/cluster/namespace/1498006988testCumulativeAck, subscription-name, 1] Closed consumer 1 +2017-06-20 18:05:55.938 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/1498006988testCumulativeAck, subscription-name, 2] Getting connection from pool +2017-06-20 18:05:55.938 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(155)- [persistent://property/cluster/namespace/1498006988testCumulativeAck, subscription-name, 2] Created consumer on broker [127.0.0.1:50166 -> 127.0.0.1:8885] +2017-06-20 18:05:55.939 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(81)- [persistent://property/cluster/namespace/1498006988testCumulativeAck, subscription-name, 1] ~ConsumerImpl +2017-06-20 18:06:00.944 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(639)- [persistent://property/cluster/namespace/1498006988testCumulativeAck, subscription-name, 2] Closed consumer 2 +2017-06-20 18:06:00.945 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/1498006988testCumulativeAck, subscription-name, 3] Getting connection from pool +2017-06-20 18:06:00.945 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(155)- [persistent://property/cluster/namespace/1498006988testCumulativeAck, subscription-name, 3] Created consumer on broker [127.0.0.1:50166 -> 127.0.0.1:8885] +2017-06-20 18:06:00.945 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(81)- [persistent://property/cluster/namespace/1498006988testCumulativeAck, subscription-name, 2] ~ConsumerImpl +2017-06-20 18:06:05.949 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(81)- [persistent://property/cluster/namespace/1498006988testCumulativeAck, subscription-name, 0] ~ConsumerImpl +2017-06-20 18:06:05.949 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(81)- [persistent://property/cluster/namespace/1498006988testCumulativeAck, subscription-name, 3] ~ConsumerImpl +2017-06-20 18:06:05.949 : WARN /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(84)- [persistent://property/cluster/namespace/1498006988testCumulativeAck, subscription-name, 3] Destroyed consumer which was not properly closed +2017-06-20 18:06:05.949 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientImpl.cc(84)- ~ClientImpl +[ OK ] BatchMessageTest.testCumulativeAck (15042 ms) +[ RUN ] BatchMessageTest.testMixedAck +2017-06-20 18:06:05.951 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConnectionPool.cc(62)- Created connection for pulsar://localhost:8885 +2017-06-20 18:06:05.952 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientConnection.cc(265)- [127.0.0.1:50169 -> 127.0.0.1:8885] Connected to broker +2017-06-20 18:06:05.953 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/BatchMessageContainer.cc(39)- { BatchContainer [size = 0] [batchSizeInBytes_ = 0] [maxAllowedMessageBatchSizeInBytes_ = 131072] [maxAllowedNumMessagesInBatch_ = 5] [topicName = persistent://property/cluster/namespace/1498006988testMixedAck] [producerName_ = ] [batchSizeInBytes_ = 0] [numberOfBatchesSent = 0] [averageBatchSize = 0]} BatchMessageContainer constructed +2017-06-20 18:06:05.953 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/1498006988testMixedAck, ] Getting connection from pool +2017-06-20 18:06:05.959 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ProducerImpl.cc(118)- [persistent://property/cluster/namespace/1498006988testMixedAck, ] Created producer on broker [127.0.0.1:50169 -> 127.0.0.1:8885] +2017-06-20 18:06:05.960 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/1498006988testMixedAck, subscription-name, 0] Getting connection from pool +2017-06-20 18:06:05.963 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(155)- [persistent://property/cluster/namespace/1498006988testMixedAck, subscription-name, 0] Created consumer on broker [127.0.0.1:50169 -> 127.0.0.1:8885] +2017-06-20 18:06:05.964 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(639)- [persistent://property/cluster/namespace/1498006988testMixedAck, subscription-name, 0] Closed consumer 0 +2017-06-20 18:06:05.964 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/1498006988testMixedAck, subscription-name, 1] Getting connection from pool +2017-06-20 18:06:05.965 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(155)- [persistent://property/cluster/namespace/1498006988testMixedAck, subscription-name, 1] Created consumer on broker [127.0.0.1:50169 -> 127.0.0.1:8885] +2017-06-20 18:06:10.980 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(639)- [persistent://property/cluster/namespace/1498006988testMixedAck, subscription-name, 1] Closed consumer 1 +2017-06-20 18:06:10.981 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/1498006988testMixedAck, subscription-name, 2] Getting connection from pool +2017-06-20 18:06:10.981 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(155)- [persistent://property/cluster/namespace/1498006988testMixedAck, subscription-name, 2] Created consumer on broker [127.0.0.1:50169 -> 127.0.0.1:8885] +2017-06-20 18:06:10.982 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(81)- [persistent://property/cluster/namespace/1498006988testMixedAck, subscription-name, 1] ~ConsumerImpl +2017-06-20 18:06:15.989 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(639)- [persistent://property/cluster/namespace/1498006988testMixedAck, subscription-name, 2] Closed consumer 2 +2017-06-20 18:06:15.990 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/1498006988testMixedAck, subscription-name, 3] Getting connection from pool +2017-06-20 18:06:15.991 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(155)- [persistent://property/cluster/namespace/1498006988testMixedAck, subscription-name, 3] Created consumer on broker [127.0.0.1:50169 -> 127.0.0.1:8885] +2017-06-20 18:06:15.991 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(81)- [persistent://property/cluster/namespace/1498006988testMixedAck, subscription-name, 2] ~ConsumerImpl +2017-06-20 18:06:20.995 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(81)- [persistent://property/cluster/namespace/1498006988testMixedAck, subscription-name, 0] ~ConsumerImpl +2017-06-20 18:06:20.995 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(81)- [persistent://property/cluster/namespace/1498006988testMixedAck, subscription-name, 3] ~ConsumerImpl +2017-06-20 18:06:20.995 : WARN /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(84)- [persistent://property/cluster/namespace/1498006988testMixedAck, subscription-name, 3] Destroyed consumer which was not properly closed +2017-06-20 18:06:20.995 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientImpl.cc(84)- ~ClientImpl +[ OK ] BatchMessageTest.testMixedAck (15047 ms) +[ RUN ] BatchMessageTest.testPermits +2017-06-20 18:06:20.997 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConnectionPool.cc(62)- Created connection for pulsar://localhost:8885 +2017-06-20 18:06:20.998 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientConnection.cc(265)- [127.0.0.1:50175 -> 127.0.0.1:8885] Connected to broker +2017-06-20 18:06:20.999 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/BatchMessageContainer.cc(39)- { BatchContainer [size = 0] [batchSizeInBytes_ = 0] [maxAllowedMessageBatchSizeInBytes_ = 131072] [maxAllowedNumMessagesInBatch_ = 10] [topicName = persistent://property/cluster/namespace/1498006988testPermits] [producerName_ = ] [batchSizeInBytes_ = 0] [numberOfBatchesSent = 0] [averageBatchSize = 0]} BatchMessageContainer constructed +2017-06-20 18:06:20.999 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/1498006988testPermits, ] Getting connection from pool +2017-06-20 18:06:21.004 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ProducerImpl.cc(118)- [persistent://property/cluster/namespace/1498006988testPermits, ] Created producer on broker [127.0.0.1:50175 -> 127.0.0.1:8885] +2017-06-20 18:06:21.005 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/1498006988testPermits, subscription-name, 0] Getting connection from pool +2017-06-20 18:06:21.008 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(155)- [persistent://property/cluster/namespace/1498006988testPermits, subscription-name, 0] Created consumer on broker [127.0.0.1:50175 -> 127.0.0.1:8885] +2017-06-20 18:06:21.009 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(639)- [persistent://property/cluster/namespace/1498006988testPermits, subscription-name, 0] Closed consumer 0 +2017-06-20 18:06:21.009 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/1498006988testPermits, subscription-name, 1] Getting connection from pool +2017-06-20 18:06:21.010 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(155)- [persistent://property/cluster/namespace/1498006988testPermits, subscription-name, 1] Created consumer on broker [127.0.0.1:50175 -> 127.0.0.1:8885] +2017-06-20 18:06:31.025 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/1498006988testPermits, ] Getting connection from pool +2017-06-20 18:06:31.026 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ProducerImpl.cc(118)- [persistent://property/cluster/namespace/1498006988testPermits, ] Created producer on broker [127.0.0.1:50175 -> 127.0.0.1:8885] +2017-06-20 18:06:41.043 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(81)- [persistent://property/cluster/namespace/1498006988testPermits, subscription-name, 0] ~ConsumerImpl +2017-06-20 18:06:41.043 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(81)- [persistent://property/cluster/namespace/1498006988testPermits, subscription-name, 1] ~ConsumerImpl +2017-06-20 18:06:41.043 : WARN /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(84)- [persistent://property/cluster/namespace/1498006988testPermits, subscription-name, 1] Destroyed consumer which was not properly closed +2017-06-20 18:06:41.043 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientImpl.cc(84)- ~ClientImpl +[ OK ] BatchMessageTest.testPermits (20047 ms) +[ RUN ] BatchMessageTest.testPartitionedTopics +2017-06-20 18:06:44.053 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConnectionPool.cc(62)- Created connection for pulsar://localhost:8885 +2017-06-20 18:06:44.054 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientConnection.cc(265)- [127.0.0.1:50182 -> 127.0.0.1:8885] Connected to broker +2017-06-20 18:06:44.056 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/BatchMessageContainer.cc(39)- { BatchContainer [size = 0] [batchSizeInBytes_ = 0] [maxAllowedMessageBatchSizeInBytes_ = 131072] [maxAllowedNumMessagesInBatch_ = 100] [topicName = persistent://property/cluster/namespace/test-partitioned-batch-messages-1498006988-partition-0] [producerName_ = ] [batchSizeInBytes_ = 0] [numberOfBatchesSent = 0] [averageBatchSize = 0]} BatchMessageContainer constructed +2017-06-20 18:06:44.056 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/BatchMessageContainer.cc(39)- { BatchContainer [size = 0] [batchSizeInBytes_ = 0] [maxAllowedMessageBatchSizeInBytes_ = 131072] [maxAllowedNumMessagesInBatch_ = 100] [topicName = persistent://property/cluster/namespace/test-partitioned-batch-messages-1498006988-partition-1] [producerName_ = ] [batchSizeInBytes_ = 0] [numberOfBatchesSent = 0] [averageBatchSize = 0]} BatchMessageContainer constructed +2017-06-20 18:06:44.056 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/BatchMessageContainer.cc(39)- { BatchContainer [size = 0] [batchSizeInBytes_ = 0] [maxAllowedMessageBatchSizeInBytes_ = 131072] [maxAllowedNumMessagesInBatch_ = 100] [topicName = persistent://property/cluster/namespace/test-partitioned-batch-messages-1498006988-partition-2] [producerName_ = ] [batchSizeInBytes_ = 0] [numberOfBatchesSent = 0] [averageBatchSize = 0]} BatchMessageContainer constructed +2017-06-20 18:06:44.056 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/BatchMessageContainer.cc(39)- { BatchContainer [size = 0] [batchSizeInBytes_ = 0] [maxAllowedMessageBatchSizeInBytes_ = 131072] [maxAllowedNumMessagesInBatch_ = 100] [topicName = persistent://property/cluster/namespace/test-partitioned-batch-messages-1498006988-partition-3] [producerName_ = ] [batchSizeInBytes_ = 0] [numberOfBatchesSent = 0] [averageBatchSize = 0]} BatchMessageContainer constructed +2017-06-20 18:06:44.056 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/BatchMessageContainer.cc(39)- { BatchContainer [size = 0] [batchSizeInBytes_ = 0] [maxAllowedMessageBatchSizeInBytes_ = 131072] [maxAllowedNumMessagesInBatch_ = 100] [topicName = persistent://property/cluster/namespace/test-partitioned-batch-messages-1498006988-partition-4] [producerName_ = ] [batchSizeInBytes_ = 0] [numberOfBatchesSent = 0] [averageBatchSize = 0]} BatchMessageContainer constructed +2017-06-20 18:06:44.056 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/BatchMessageContainer.cc(39)- { BatchContainer [size = 0] [batchSizeInBytes_ = 0] [maxAllowedMessageBatchSizeInBytes_ = 131072] [maxAllowedNumMessagesInBatch_ = 100] [topicName = persistent://property/cluster/namespace/test-partitioned-batch-messages-1498006988-partition-5] [producerName_ = ] [batchSizeInBytes_ = 0] [numberOfBatchesSent = 0] [averageBatchSize = 0]} BatchMessageContainer constructed +2017-06-20 18:06:44.056 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/BatchMessageContainer.cc(39)- { BatchContainer [size = 0] [batchSizeInBytes_ = 0] [maxAllowedMessageBatchSizeInBytes_ = 131072] [maxAllowedNumMessagesInBatch_ = 100] [topicName = persistent://property/cluster/namespace/test-partitioned-batch-messages-1498006988-partition-6] [producerName_ = ] [batchSizeInBytes_ = 0] [numberOfBatchesSent = 0] [averageBatchSize = 0]} BatchMessageContainer constructed +2017-06-20 18:06:44.056 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/test-partitioned-batch-messages-1498006988-partition-0, ] Getting connection from pool +2017-06-20 18:06:44.056 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/test-partitioned-batch-messages-1498006988-partition-1, ] Getting connection from pool +2017-06-20 18:06:44.056 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/test-partitioned-batch-messages-1498006988-partition-2, ] Getting connection from pool +2017-06-20 18:06:44.056 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/test-partitioned-batch-messages-1498006988-partition-3, ] Getting connection from pool +2017-06-20 18:06:44.056 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/test-partitioned-batch-messages-1498006988-partition-4, ] Getting connection from pool +2017-06-20 18:06:44.056 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/test-partitioned-batch-messages-1498006988-partition-5, ] Getting connection from pool +2017-06-20 18:06:44.057 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/test-partitioned-batch-messages-1498006988-partition-6, ] Getting connection from pool +2017-06-20 18:06:44.065 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ProducerImpl.cc(118)- [persistent://property/cluster/namespace/test-partitioned-batch-messages-1498006988-partition-2, ] Created producer on broker [127.0.0.1:50182 -> 127.0.0.1:8885] +2017-06-20 18:06:44.065 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ProducerImpl.cc(118)- [persistent://property/cluster/namespace/test-partitioned-batch-messages-1498006988-partition-0, ] Created producer on broker [127.0.0.1:50182 -> 127.0.0.1:8885] +2017-06-20 18:06:44.065 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ProducerImpl.cc(118)- [persistent://property/cluster/namespace/test-partitioned-batch-messages-1498006988-partition-1, ] Created producer on broker [127.0.0.1:50182 -> 127.0.0.1:8885] +2017-06-20 18:06:44.065 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ProducerImpl.cc(118)- [persistent://property/cluster/namespace/test-partitioned-batch-messages-1498006988-partition-3, ] Created producer on broker [127.0.0.1:50182 -> 127.0.0.1:8885] +2017-06-20 18:06:44.066 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ProducerImpl.cc(118)- [persistent://property/cluster/namespace/test-partitioned-batch-messages-1498006988-partition-5, ] Created producer on broker [127.0.0.1:50182 -> 127.0.0.1:8885] +2017-06-20 18:06:44.066 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ProducerImpl.cc(118)- [persistent://property/cluster/namespace/test-partitioned-batch-messages-1498006988-partition-6, ] Created producer on broker [127.0.0.1:50182 -> 127.0.0.1:8885] +2017-06-20 18:06:44.066 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ProducerImpl.cc(118)- [persistent://property/cluster/namespace/test-partitioned-batch-messages-1498006988-partition-4, ] Created producer on broker [127.0.0.1:50182 -> 127.0.0.1:8885] +2017-06-20 18:06:44.066 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/test-partitioned-batch-messages-1498006988-partition-0, subscription-name0, 0] Getting connection from pool +2017-06-20 18:06:44.066 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/test-partitioned-batch-messages-1498006988-partition-1, subscription-name1, 1] Getting connection from pool +2017-06-20 18:06:44.066 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/test-partitioned-batch-messages-1498006988-partition-2, subscription-name2, 2] Getting connection from pool +2017-06-20 18:06:44.067 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/test-partitioned-batch-messages-1498006988-partition-3, subscription-name3, 3] Getting connection from pool +2017-06-20 18:06:44.067 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/test-partitioned-batch-messages-1498006988-partition-4, subscription-name4, 4] Getting connection from pool +2017-06-20 18:06:44.067 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/test-partitioned-batch-messages-1498006988-partition-5, subscription-name5, 5] Getting connection from pool +2017-06-20 18:06:44.067 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/test-partitioned-batch-messages-1498006988-partition-6, subscription-name6, 6] Getting connection from pool +2017-06-20 18:06:44.073 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(155)- [persistent://property/cluster/namespace/test-partitioned-batch-messages-1498006988-partition-1, subscription-name1, 1] Created consumer on broker [127.0.0.1:50182 -> 127.0.0.1:8885] +2017-06-20 18:06:44.073 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(155)- [persistent://property/cluster/namespace/test-partitioned-batch-messages-1498006988-partition-0, subscription-name0, 0] Created consumer on broker [127.0.0.1:50182 -> 127.0.0.1:8885] +2017-06-20 18:06:44.073 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(155)- [persistent://property/cluster/namespace/test-partitioned-batch-messages-1498006988-partition-3, subscription-name3, 3] Created consumer on broker [127.0.0.1:50182 -> 127.0.0.1:8885] +2017-06-20 18:06:44.074 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(155)- [persistent://property/cluster/namespace/test-partitioned-batch-messages-1498006988-partition-5, subscription-name5, 5] Created consumer on broker [127.0.0.1:50182 -> 127.0.0.1:8885] +2017-06-20 18:06:44.074 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(155)- [persistent://property/cluster/namespace/test-partitioned-batch-messages-1498006988-partition-4, subscription-name4, 4] Created consumer on broker [127.0.0.1:50182 -> 127.0.0.1:8885] +2017-06-20 18:06:44.074 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(155)- [persistent://property/cluster/namespace/test-partitioned-batch-messages-1498006988-partition-6, subscription-name6, 6] Created consumer on broker [127.0.0.1:50182 -> 127.0.0.1:8885] +2017-06-20 18:06:44.074 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(155)- [persistent://property/cluster/namespace/test-partitioned-batch-messages-1498006988-partition-2, subscription-name2, 2] Created consumer on broker [127.0.0.1:50182 -> 127.0.0.1:8885] +2017-06-20 18:06:44.074 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/PartitionedConsumerImpl.cc(222)- Successfully Subscribed to Partitioned Topic - persistent://property/cluster/namespace/test-partitioned-batch-messages-1498006988 with - 7 Partitions. +[ OK ] BatchMessageTest.testPartitionedTopics (33174 ms) +[----------] 9 tests from BatchMessageTest (159958 ms total) + +[----------] 1 test from BinaryLookupServiceTest +[ RUN ] BinaryLookupServiceTest.basicLookup +2017-06-20 18:07:14.219 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConnectionPool.cc(62)- Created connection for pulsar://localhost:8885 +2017-06-20 18:07:14.220 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientConnection.cc(265)- [127.0.0.1:50188 -> 127.0.0.1:8885] Connected to broker +[ OK ] BinaryLookupServiceTest.basicLookup (4 ms) +[----------] 1 test from BinaryLookupServiceTest (4 ms total) + +[----------] 6 tests from BlockingQueueTest +[ RUN ] BlockingQueueTest.testBasic +[ OK ] BlockingQueueTest.testBasic (1 ms) +[ RUN ] BlockingQueueTest.testQueueOperations +[ OK ] BlockingQueueTest.testQueueOperations (0 ms) +[ RUN ] BlockingQueueTest.testBlockingProducer +[ OK ] BlockingQueueTest.testBlockingProducer (0 ms) +[ RUN ] BlockingQueueTest.testBlockingConsumer +[ OK ] BlockingQueueTest.testBlockingConsumer (0 ms) +[ RUN ] BlockingQueueTest.testTimeout +[ OK ] BlockingQueueTest.testTimeout (3008 ms) +[ RUN ] BlockingQueueTest.testReservedSpot +[ OK ] BlockingQueueTest.testReservedSpot (0 ms) +[----------] 6 tests from BlockingQueueTest (3009 ms total) + +[----------] 2 tests from ClientTest +[ RUN ] ClientTest.testChecksumComputation +[ OK ] ClientTest.testChecksumComputation (0 ms) +[ RUN ] ClientTest.testSwHwChecksum +[ OK ] ClientTest.testSwHwChecksum (0 ms) +[----------] 2 tests from ClientTest (0 ms total) + +[----------] 4 tests from ConsumerStatsTest +[ RUN ] ConsumerStatsTest.testBacklogInfo +2017-06-20 18:07:17.231 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(127)- Curl Lookup Request sent forhttp://localhost:8765/admin/persistent/property/cluster/namespace/testBacklogInfo-1498007237/partitions +2017-06-20 18:07:17.234 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(139)- Response received for url http://localhost:8765/admin/persistent/property/cluster/namespace/testBacklogInfo-1498007237/partitions code 200 +2017-06-20 18:07:17.234 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(179)- parsePartitionData = { LookupDataResult [brokerUrl_ = ] [brokerUrlSsl_ = ] [partitions = 0] [authoritative = 0] [redirect = 0] +2017-06-20 18:07:17.234 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/testBacklogInfo-1498007237, subscription-name, 0] Getting connection from pool +2017-06-20 18:07:17.234 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(127)- Curl Lookup Request sent forhttp://localhost:8765/lookup/v2/destination/persistent/property/cluster/namespace/testBacklogInfo-1498007237 +2017-06-20 18:07:17.236 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(139)- Response received for url http://localhost:8765/lookup/v2/destination/persistent/property/cluster/namespace/testBacklogInfo-1498007237 code 200 +2017-06-20 18:07:17.236 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(207)- parseLookupData = { LookupDataResult [brokerUrl_ = pulsar://localhost:8885] [brokerUrlSsl_ = ] [partitions = 0] [authoritative = 0] [redirect = 0] +2017-06-20 18:07:17.237 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConnectionPool.cc(62)- Created connection for pulsar://localhost:8885 +2017-06-20 18:07:17.238 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientConnection.cc(265)- [127.0.0.1:50195 -> 127.0.0.1:8885] Connected to broker +2017-06-20 18:07:17.247 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(155)- [persistent://property/cluster/namespace/testBacklogInfo-1498007237, subscription-name, 0] Created consumer on broker [127.0.0.1:50195 -> 127.0.0.1:8885] +2017-06-20 18:07:17.247 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(208)- [persistent://property/cluster/namespace/testBacklogInfo-1498007237, subscription-name, 0] Unsubscribing +2017-06-20 18:07:17.252 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(240)- [persistent://property/cluster/namespace/testBacklogInfo-1498007237, subscription-name, 0] Unsubscribed successfully +2017-06-20 18:07:17.252 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(127)- Curl Lookup Request sent forhttp://localhost:8765/admin/persistent/property/cluster/namespace/testBacklogInfo-1498007237/partitions +2017-06-20 18:07:17.255 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(139)- Response received for url http://localhost:8765/admin/persistent/property/cluster/namespace/testBacklogInfo-1498007237/partitions code 200 +2017-06-20 18:07:17.255 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(179)- parsePartitionData = { LookupDataResult [brokerUrl_ = ] [brokerUrlSsl_ = ] [partitions = 0] [authoritative = 0] [redirect = 0] +2017-06-20 18:07:17.255 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/testBacklogInfo-1498007237, subscription-name, 1] Getting connection from pool +2017-06-20 18:07:17.255 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(127)- Curl Lookup Request sent forhttp://localhost:8765/lookup/v2/destination/persistent/property/cluster/namespace/testBacklogInfo-1498007237 +2017-06-20 18:07:17.257 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(139)- Response received for url http://localhost:8765/lookup/v2/destination/persistent/property/cluster/namespace/testBacklogInfo-1498007237 code 200 +2017-06-20 18:07:17.257 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(207)- parseLookupData = { LookupDataResult [brokerUrl_ = pulsar://localhost:8885] [brokerUrlSsl_ = ] [partitions = 0] [authoritative = 0] [redirect = 0] +2017-06-20 18:07:17.260 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(155)- [persistent://property/cluster/namespace/testBacklogInfo-1498007237, subscription-name, 1] Created consumer on broker [127.0.0.1:50195 -> 127.0.0.1:8885] +2017-06-20 18:07:17.260 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(127)- Curl Lookup Request sent forhttp://localhost:8765/admin/persistent/property/cluster/namespace/testBacklogInfo-1498007237/partitions +2017-06-20 18:07:17.262 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(139)- Response received for url http://localhost:8765/admin/persistent/property/cluster/namespace/testBacklogInfo-1498007237/partitions code 200 +2017-06-20 18:07:17.262 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(179)- parsePartitionData = { LookupDataResult [brokerUrl_ = ] [brokerUrlSsl_ = ] [partitions = 0] [authoritative = 0] [redirect = 0] +2017-06-20 18:07:17.262 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/testBacklogInfo-1498007237, ] Getting connection from pool +2017-06-20 18:07:17.262 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(127)- Curl Lookup Request sent forhttp://localhost:8765/lookup/v2/destination/persistent/property/cluster/namespace/testBacklogInfo-1498007237 +2017-06-20 18:07:17.264 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(139)- Response received for url http://localhost:8765/lookup/v2/destination/persistent/property/cluster/namespace/testBacklogInfo-1498007237 code 200 +2017-06-20 18:07:17.264 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(207)- parseLookupData = { LookupDataResult [brokerUrl_ = pulsar://localhost:8885] [brokerUrlSsl_ = ] [partitions = 0] [authoritative = 0] [redirect = 0] +2017-06-20 18:07:17.265 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ProducerImpl.cc(118)- [persistent://property/cluster/namespace/testBacklogInfo-1498007237, ] Created producer on broker [127.0.0.1:50195 -> 127.0.0.1:8885] +2017-06-20 18:07:20.800 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(208)- [persistent://property/cluster/namespace/testBacklogInfo-1498007237, subscription-name, 1] Unsubscribing +2017-06-20 18:07:20.804 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(240)- [persistent://property/cluster/namespace/testBacklogInfo-1498007237, subscription-name, 1] Unsubscribed successfully +2017-06-20 18:07:20.804 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(81)- [persistent://property/cluster/namespace/testBacklogInfo-1498007237, subscription-name, 0] ~ConsumerImpl +2017-06-20 18:07:20.804 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(81)- [persistent://property/cluster/namespace/testBacklogInfo-1498007237, subscription-name, 1] ~ConsumerImpl +2017-06-20 18:07:20.804 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientImpl.cc(84)- ~ClientImpl +[ OK ] ConsumerStatsTest.testBacklogInfo (3574 ms) +[ RUN ] ConsumerStatsTest.testFailure +2017-06-20 18:07:20.805 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(127)- Curl Lookup Request sent forhttp://localhost:8765/admin/persistent/property/cluster/namespace/testFailure-1498007240/partitions +2017-06-20 18:07:20.807 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(139)- Response received for url http://localhost:8765/admin/persistent/property/cluster/namespace/testFailure-1498007240/partitions code 200 +2017-06-20 18:07:20.807 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(179)- parsePartitionData = { LookupDataResult [brokerUrl_ = ] [brokerUrlSsl_ = ] [partitions = 0] [authoritative = 0] [redirect = 0] +2017-06-20 18:07:20.808 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/testFailure-1498007240, subscription-name, 0] Getting connection from pool +2017-06-20 18:07:20.808 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(127)- Curl Lookup Request sent forhttp://localhost:8765/lookup/v2/destination/persistent/property/cluster/namespace/testFailure-1498007240 +2017-06-20 18:07:20.810 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(139)- Response received for url http://localhost:8765/lookup/v2/destination/persistent/property/cluster/namespace/testFailure-1498007240 code 200 +2017-06-20 18:07:20.810 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(207)- parseLookupData = { LookupDataResult [brokerUrl_ = pulsar://localhost:8885] [brokerUrlSsl_ = ] [partitions = 0] [authoritative = 0] [redirect = 0] +2017-06-20 18:07:20.811 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConnectionPool.cc(62)- Created connection for pulsar://localhost:8885 +2017-06-20 18:07:20.812 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientConnection.cc(265)- [127.0.0.1:50211 -> 127.0.0.1:8885] Connected to broker +2017-06-20 18:07:20.818 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(155)- [persistent://property/cluster/namespace/testFailure-1498007240, subscription-name, 0] Created consumer on broker [127.0.0.1:50211 -> 127.0.0.1:8885] +2017-06-20 18:07:20.818 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(208)- [persistent://property/cluster/namespace/testFailure-1498007240, subscription-name, 0] Unsubscribing +2017-06-20 18:07:20.820 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(240)- [persistent://property/cluster/namespace/testFailure-1498007240, subscription-name, 0] Unsubscribed successfully +2017-06-20 18:07:20.820 : ERROR /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(724)- [persistent://property/cluster/namespace/testFailure-1498007240, subscription-name, 0] Client connection is not open, please try again later. +2017-06-20 18:07:20.820 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(127)- Curl Lookup Request sent forhttp://localhost:8765/admin/persistent/property/cluster/namespace/testFailure-1498007240/partitions +2017-06-20 18:07:20.822 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(139)- Response received for url http://localhost:8765/admin/persistent/property/cluster/namespace/testFailure-1498007240/partitions code 200 +2017-06-20 18:07:20.822 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(179)- parsePartitionData = { LookupDataResult [brokerUrl_ = ] [brokerUrlSsl_ = ] [partitions = 0] [authoritative = 0] [redirect = 0] +2017-06-20 18:07:20.822 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/testFailure-1498007240, subscription-name, 1] Getting connection from pool +2017-06-20 18:07:20.822 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(127)- Curl Lookup Request sent forhttp://localhost:8765/lookup/v2/destination/persistent/property/cluster/namespace/testFailure-1498007240 +2017-06-20 18:07:20.824 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(139)- Response received for url http://localhost:8765/lookup/v2/destination/persistent/property/cluster/namespace/testFailure-1498007240 code 200 +2017-06-20 18:07:20.824 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(207)- parseLookupData = { LookupDataResult [brokerUrl_ = pulsar://localhost:8885] [brokerUrlSsl_ = ] [partitions = 0] [authoritative = 0] [redirect = 0] +2017-06-20 18:07:20.827 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(155)- [persistent://property/cluster/namespace/testFailure-1498007240, subscription-name, 1] Created consumer on broker [127.0.0.1:50211 -> 127.0.0.1:8885] +2017-06-20 18:07:20.827 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(127)- Curl Lookup Request sent forhttp://localhost:8765/admin/persistent/property/cluster/namespace/testFailure-1498007240/partitions +2017-06-20 18:07:20.830 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(139)- Response received for url http://localhost:8765/admin/persistent/property/cluster/namespace/testFailure-1498007240/partitions code 200 +2017-06-20 18:07:20.830 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(179)- parsePartitionData = { LookupDataResult [brokerUrl_ = ] [brokerUrlSsl_ = ] [partitions = 0] [authoritative = 0] [redirect = 0] +2017-06-20 18:07:20.830 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/testFailure-1498007240, ] Getting connection from pool +2017-06-20 18:07:20.830 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(127)- Curl Lookup Request sent forhttp://localhost:8765/lookup/v2/destination/persistent/property/cluster/namespace/testFailure-1498007240 +2017-06-20 18:07:20.832 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(139)- Response received for url http://localhost:8765/lookup/v2/destination/persistent/property/cluster/namespace/testFailure-1498007240 code 200 +2017-06-20 18:07:20.832 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(207)- parseLookupData = { LookupDataResult [brokerUrl_ = pulsar://localhost:8885] [brokerUrlSsl_ = ] [partitions = 0] [authoritative = 0] [redirect = 0] +2017-06-20 18:07:20.833 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ProducerImpl.cc(118)- [persistent://property/cluster/namespace/testFailure-1498007240, ] Created producer on broker [127.0.0.1:50211 -> 127.0.0.1:8885] +2017-06-20 18:07:20.841 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(208)- [persistent://property/cluster/namespace/testFailure-1498007240, subscription-name, 1] Unsubscribing +2017-06-20 18:07:20.844 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(240)- [persistent://property/cluster/namespace/testFailure-1498007240, subscription-name, 1] Unsubscribed successfully +2017-06-20 18:07:20.844 : ERROR /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(724)- [persistent://property/cluster/namespace/testFailure-1498007240, subscription-name, 1] Client connection is not open, please try again later. +2017-06-20 18:07:20.844 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(81)- [persistent://property/cluster/namespace/testFailure-1498007240, subscription-name, 0] ~ConsumerImpl +2017-06-20 18:07:20.844 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(81)- [persistent://property/cluster/namespace/testFailure-1498007240, subscription-name, 1] ~ConsumerImpl +2017-06-20 18:07:20.844 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientImpl.cc(84)- ~ClientImpl +[ OK ] ConsumerStatsTest.testFailure (39 ms) +[ RUN ] ConsumerStatsTest.testCachingMechanism +2017-06-20 18:07:20.845 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(127)- Curl Lookup Request sent forhttp://localhost:8765/admin/persistent/property/cluster/namespace/testCachingMechanism-1498007240/partitions +2017-06-20 18:07:20.847 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(139)- Response received for url http://localhost:8765/admin/persistent/property/cluster/namespace/testCachingMechanism-1498007240/partitions code 200 +2017-06-20 18:07:20.847 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(179)- parsePartitionData = { LookupDataResult [brokerUrl_ = ] [brokerUrlSsl_ = ] [partitions = 0] [authoritative = 0] [redirect = 0] +2017-06-20 18:07:20.847 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/testCachingMechanism-1498007240, subscription-name, 0] Getting connection from pool +2017-06-20 18:07:20.847 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(127)- Curl Lookup Request sent forhttp://localhost:8765/lookup/v2/destination/persistent/property/cluster/namespace/testCachingMechanism-1498007240 +2017-06-20 18:07:20.849 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(139)- Response received for url http://localhost:8765/lookup/v2/destination/persistent/property/cluster/namespace/testCachingMechanism-1498007240 code 200 +2017-06-20 18:07:20.849 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(207)- parseLookupData = { LookupDataResult [brokerUrl_ = pulsar://localhost:8885] [brokerUrlSsl_ = ] [partitions = 0] [authoritative = 0] [redirect = 0] +2017-06-20 18:07:20.850 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConnectionPool.cc(62)- Created connection for pulsar://localhost:8885 +2017-06-20 18:07:20.851 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientConnection.cc(265)- [127.0.0.1:50225 -> 127.0.0.1:8885] Connected to broker +2017-06-20 18:07:20.857 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(155)- [persistent://property/cluster/namespace/testCachingMechanism-1498007240, subscription-name, 0] Created consumer on broker [127.0.0.1:50225 -> 127.0.0.1:8885] +2017-06-20 18:07:20.857 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(208)- [persistent://property/cluster/namespace/testCachingMechanism-1498007240, subscription-name, 0] Unsubscribing +2017-06-20 18:07:20.860 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(240)- [persistent://property/cluster/namespace/testCachingMechanism-1498007240, subscription-name, 0] Unsubscribed successfully +2017-06-20 18:07:20.860 : ERROR /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(724)- [persistent://property/cluster/namespace/testCachingMechanism-1498007240, subscription-name, 0] Client connection is not open, please try again later. +2017-06-20 18:07:20.860 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(127)- Curl Lookup Request sent forhttp://localhost:8765/admin/persistent/property/cluster/namespace/testCachingMechanism-1498007240/partitions +2017-06-20 18:07:20.863 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(139)- Response received for url http://localhost:8765/admin/persistent/property/cluster/namespace/testCachingMechanism-1498007240/partitions code 200 +2017-06-20 18:07:20.863 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(179)- parsePartitionData = { LookupDataResult [brokerUrl_ = ] [brokerUrlSsl_ = ] [partitions = 0] [authoritative = 0] [redirect = 0] +2017-06-20 18:07:20.863 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/testCachingMechanism-1498007240, subscription-name, 1] Getting connection from pool +2017-06-20 18:07:20.863 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(127)- Curl Lookup Request sent forhttp://localhost:8765/lookup/v2/destination/persistent/property/cluster/namespace/testCachingMechanism-1498007240 +2017-06-20 18:07:20.864 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(139)- Response received for url http://localhost:8765/lookup/v2/destination/persistent/property/cluster/namespace/testCachingMechanism-1498007240 code 200 +2017-06-20 18:07:20.864 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(207)- parseLookupData = { LookupDataResult [brokerUrl_ = pulsar://localhost:8885] [brokerUrlSsl_ = ] [partitions = 0] [authoritative = 0] [redirect = 0] +2017-06-20 18:07:20.868 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(155)- [persistent://property/cluster/namespace/testCachingMechanism-1498007240, subscription-name, 1] Created consumer on broker [127.0.0.1:50225 -> 127.0.0.1:8885] +2017-06-20 18:07:20.868 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(127)- Curl Lookup Request sent forhttp://localhost:8765/admin/persistent/property/cluster/namespace/testCachingMechanism-1498007240/partitions +2017-06-20 18:07:20.869 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(139)- Response received for url http://localhost:8765/admin/persistent/property/cluster/namespace/testCachingMechanism-1498007240/partitions code 200 +2017-06-20 18:07:20.869 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(179)- parsePartitionData = { LookupDataResult [brokerUrl_ = ] [brokerUrlSsl_ = ] [partitions = 0] [authoritative = 0] [redirect = 0] +2017-06-20 18:07:20.869 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/testCachingMechanism-1498007240, ] Getting connection from pool +2017-06-20 18:07:20.870 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(127)- Curl Lookup Request sent forhttp://localhost:8765/lookup/v2/destination/persistent/property/cluster/namespace/testCachingMechanism-1498007240 +2017-06-20 18:07:20.871 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(139)- Response received for url http://localhost:8765/lookup/v2/destination/persistent/property/cluster/namespace/testCachingMechanism-1498007240 code 200 +2017-06-20 18:07:20.871 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(207)- parseLookupData = { LookupDataResult [brokerUrl_ = pulsar://localhost:8885] [brokerUrlSsl_ = ] [partitions = 0] [authoritative = 0] [redirect = 0] +2017-06-20 18:07:20.872 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ProducerImpl.cc(118)- [persistent://property/cluster/namespace/testCachingMechanism-1498007240, ] Created producer on broker [127.0.0.1:50225 -> 127.0.0.1:8885] +2017-06-20 18:07:24.894 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(208)- [persistent://property/cluster/namespace/testCachingMechanism-1498007240, subscription-name, 1] Unsubscribing +2017-06-20 18:07:24.899 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(240)- [persistent://property/cluster/namespace/testCachingMechanism-1498007240, subscription-name, 1] Unsubscribed successfully +2017-06-20 18:07:24.899 : ERROR /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(724)- [persistent://property/cluster/namespace/testCachingMechanism-1498007240, subscription-name, 1] Client connection is not open, please try again later. +2017-06-20 18:07:24.899 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(81)- [persistent://property/cluster/namespace/testCachingMechanism-1498007240, subscription-name, 0] ~ConsumerImpl +2017-06-20 18:07:24.899 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(81)- [persistent://property/cluster/namespace/testCachingMechanism-1498007240, subscription-name, 1] ~ConsumerImpl +2017-06-20 18:07:24.899 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientImpl.cc(84)- ~ClientImpl +[ OK ] ConsumerStatsTest.testCachingMechanism (4055 ms) +[ RUN ] ConsumerStatsTest.testAsyncCallOnPartitionedTopic +2017-06-20 18:07:25.903 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/tests/ConsumerStatsTest.cc(241)- res = 204 +2017-06-20 18:07:25.903 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(127)- Curl Lookup Request sent forhttp://localhost:8765/admin/persistent/property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244/partitions +2017-06-20 18:07:25.906 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(139)- Response received for url http://localhost:8765/admin/persistent/property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244/partitions code 200 +2017-06-20 18:07:25.906 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(179)- parsePartitionData = { LookupDataResult [brokerUrl_ = ] [brokerUrlSsl_ = ] [partitions = 7] [authoritative = 0] [redirect = 0] +2017-06-20 18:07:25.906 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-0, subscription-name0, 0] Getting connection from pool +2017-06-20 18:07:25.906 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-1, subscription-name1, 1] Getting connection from pool +2017-06-20 18:07:25.906 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-2, subscription-name2, 2] Getting connection from pool +2017-06-20 18:07:25.906 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-3, subscription-name3, 3] Getting connection from pool +2017-06-20 18:07:25.906 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-4, subscription-name4, 4] Getting connection from pool +2017-06-20 18:07:25.906 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-5, subscription-name5, 5] Getting connection from pool +2017-06-20 18:07:25.906 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-6, subscription-name6, 6] Getting connection from pool +2017-06-20 18:07:25.907 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(127)- Curl Lookup Request sent forhttp://localhost:8765/lookup/v2/destination/persistent/property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-0 +2017-06-20 18:07:25.909 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(139)- Response received for url http://localhost:8765/lookup/v2/destination/persistent/property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-0 code 200 +2017-06-20 18:07:25.909 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(207)- parseLookupData = { LookupDataResult [brokerUrl_ = pulsar://localhost:8885] [brokerUrlSsl_ = ] [partitions = 0] [authoritative = 0] [redirect = 0] +2017-06-20 18:07:25.910 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConnectionPool.cc(62)- Created connection for pulsar://localhost:8885 +2017-06-20 18:07:25.910 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(127)- Curl Lookup Request sent forhttp://localhost:8765/lookup/v2/destination/persistent/property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-1 +2017-06-20 18:07:25.911 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ClientConnection.cc(265)- [127.0.0.1:50245 -> 127.0.0.1:8885] Connected to broker +2017-06-20 18:07:25.912 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(139)- Response received for url http://localhost:8765/lookup/v2/destination/persistent/property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-1 code 200 +2017-06-20 18:07:25.912 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(207)- parseLookupData = { LookupDataResult [brokerUrl_ = pulsar://localhost:8885] [brokerUrlSsl_ = ] [partitions = 0] [authoritative = 0] [redirect = 0] +2017-06-20 18:07:25.913 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(127)- Curl Lookup Request sent forhttp://localhost:8765/lookup/v2/destination/persistent/property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-2 +2017-06-20 18:07:25.915 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(139)- Response received for url http://localhost:8765/lookup/v2/destination/persistent/property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-2 code 200 +2017-06-20 18:07:25.915 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(207)- parseLookupData = { LookupDataResult [brokerUrl_ = pulsar://localhost:8885] [brokerUrlSsl_ = ] [partitions = 0] [authoritative = 0] [redirect = 0] +2017-06-20 18:07:25.916 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(127)- Curl Lookup Request sent forhttp://localhost:8765/lookup/v2/destination/persistent/property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-3 +2017-06-20 18:07:25.918 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(139)- Response received for url http://localhost:8765/lookup/v2/destination/persistent/property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-3 code 200 +2017-06-20 18:07:25.918 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(207)- parseLookupData = { LookupDataResult [brokerUrl_ = pulsar://localhost:8885] [brokerUrlSsl_ = ] [partitions = 0] [authoritative = 0] [redirect = 0] +2017-06-20 18:07:25.918 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(127)- Curl Lookup Request sent forhttp://localhost:8765/lookup/v2/destination/persistent/property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-4 +2017-06-20 18:07:25.920 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(139)- Response received for url http://localhost:8765/lookup/v2/destination/persistent/property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-4 code 200 +2017-06-20 18:07:25.920 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(207)- parseLookupData = { LookupDataResult [brokerUrl_ = pulsar://localhost:8885] [brokerUrlSsl_ = ] [partitions = 0] [authoritative = 0] [redirect = 0] +2017-06-20 18:07:25.920 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(127)- Curl Lookup Request sent forhttp://localhost:8765/lookup/v2/destination/persistent/property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-5 +2017-06-20 18:07:25.923 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(155)- [persistent://property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-0, subscription-name0, 0] Created consumer on broker [127.0.0.1:50245 -> 127.0.0.1:8885] +2017-06-20 18:07:25.923 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(139)- Response received for url http://localhost:8765/lookup/v2/destination/persistent/property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-5 code 200 +2017-06-20 18:07:25.923 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(207)- parseLookupData = { LookupDataResult [brokerUrl_ = pulsar://localhost:8885] [brokerUrlSsl_ = ] [partitions = 0] [authoritative = 0] [redirect = 0] +2017-06-20 18:07:25.923 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(127)- Curl Lookup Request sent forhttp://localhost:8765/lookup/v2/destination/persistent/property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-6 +2017-06-20 18:07:25.924 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(155)- [persistent://property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-1, subscription-name1, 1] Created consumer on broker [127.0.0.1:50245 -> 127.0.0.1:8885] +2017-06-20 18:07:25.926 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(155)- [persistent://property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-2, subscription-name2, 2] Created consumer on broker [127.0.0.1:50245 -> 127.0.0.1:8885] +2017-06-20 18:07:25.926 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(139)- Response received for url http://localhost:8765/lookup/v2/destination/persistent/property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-6 code 200 +2017-06-20 18:07:25.926 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(207)- parseLookupData = { LookupDataResult [brokerUrl_ = pulsar://localhost:8885] [brokerUrlSsl_ = ] [partitions = 0] [authoritative = 0] [redirect = 0] +2017-06-20 18:07:25.929 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(155)- [persistent://property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-3, subscription-name3, 3] Created consumer on broker [127.0.0.1:50245 -> 127.0.0.1:8885] +2017-06-20 18:07:25.931 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(155)- [persistent://property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-4, subscription-name4, 4] Created consumer on broker [127.0.0.1:50245 -> 127.0.0.1:8885] +2017-06-20 18:07:25.932 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(155)- [persistent://property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-5, subscription-name5, 5] Created consumer on broker [127.0.0.1:50245 -> 127.0.0.1:8885] +2017-06-20 18:07:25.933 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(155)- [persistent://property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-6, subscription-name6, 6] Created consumer on broker [127.0.0.1:50245 -> 127.0.0.1:8885] +2017-06-20 18:07:25.933 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/PartitionedConsumerImpl.cc(222)- Successfully Subscribed to Partitioned Topic - persistent://property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244 with - 7 Partitions. +2017-06-20 18:07:25.934 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/PartitionedConsumerImpl.cc(102)- [persistent://property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244,subscription-name] Unsubscribing +2017-06-20 18:07:25.934 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(208)- [persistent://property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-0, subscription-name0, 0] Unsubscribing +2017-06-20 18:07:25.934 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(208)- [persistent://property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-1, subscription-name1, 1] Unsubscribing +2017-06-20 18:07:25.934 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(208)- [persistent://property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-2, subscription-name2, 2] Unsubscribing +2017-06-20 18:07:25.934 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(208)- [persistent://property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-3, subscription-name3, 3] Unsubscribing +2017-06-20 18:07:25.934 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(208)- [persistent://property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-4, subscription-name4, 4] Unsubscribing +2017-06-20 18:07:25.934 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(208)- [persistent://property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-5, subscription-name5, 5] Unsubscribing +2017-06-20 18:07:25.934 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(208)- [persistent://property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-6, subscription-name6, 6] Unsubscribing +2017-06-20 18:07:25.945 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(240)- [persistent://property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-0, subscription-name0, 0] Unsubscribed successfully +2017-06-20 18:07:25.945 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/PartitionedConsumerImpl.cc(146)- Successfully Unsubscribed Consumer - 0 for Subscription - subscription-name for Topic - persistent://property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244 +2017-06-20 18:07:25.945 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(240)- [persistent://property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-1, subscription-name1, 1] Unsubscribed successfully +2017-06-20 18:07:25.945 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/PartitionedConsumerImpl.cc(146)- Successfully Unsubscribed Consumer - 1 for Subscription - subscription-name for Topic - persistent://property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244 +2017-06-20 18:07:25.945 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(240)- [persistent://property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-2, subscription-name2, 2] Unsubscribed successfully +2017-06-20 18:07:25.945 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/PartitionedConsumerImpl.cc(146)- Successfully Unsubscribed Consumer - 2 for Subscription - subscription-name for Topic - persistent://property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244 +2017-06-20 18:07:25.945 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(240)- [persistent://property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-3, subscription-name3, 3] Unsubscribed successfully +2017-06-20 18:07:25.945 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/PartitionedConsumerImpl.cc(146)- Successfully Unsubscribed Consumer - 3 for Subscription - subscription-name for Topic - persistent://property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244 +2017-06-20 18:07:25.945 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(240)- [persistent://property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-4, subscription-name4, 4] Unsubscribed successfully +2017-06-20 18:07:25.945 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/PartitionedConsumerImpl.cc(146)- Successfully Unsubscribed Consumer - 4 for Subscription - subscription-name for Topic - persistent://property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244 +2017-06-20 18:07:25.946 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(240)- [persistent://property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-5, subscription-name5, 5] Unsubscribed successfully +2017-06-20 18:07:25.946 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/PartitionedConsumerImpl.cc(146)- Successfully Unsubscribed Consumer - 5 for Subscription - subscription-name for Topic - persistent://property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244 +2017-06-20 18:07:25.947 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(240)- [persistent://property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-6, subscription-name6, 6] Unsubscribed successfully +2017-06-20 18:07:25.947 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/PartitionedConsumerImpl.cc(146)- Successfully Unsubscribed Consumer - 6 for Subscription - subscription-name for Topic - persistent://property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244 +2017-06-20 18:07:25.947 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(127)- Curl Lookup Request sent forhttp://localhost:8765/admin/persistent/property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244/partitions +2017-06-20 18:07:25.949 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(139)- Response received for url http://localhost:8765/admin/persistent/property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244/partitions code 200 +2017-06-20 18:07:25.949 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(179)- parsePartitionData = { LookupDataResult [brokerUrl_ = ] [brokerUrlSsl_ = ] [partitions = 7] [authoritative = 0] [redirect = 0] +2017-06-20 18:07:25.949 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-0, subscription-name0, 7] Getting connection from pool +2017-06-20 18:07:25.949 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-1, subscription-name1, 8] Getting connection from pool +2017-06-20 18:07:25.949 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-2, subscription-name2, 9] Getting connection from pool +2017-06-20 18:07:25.949 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-3, subscription-name3, 10] Getting connection from pool +2017-06-20 18:07:25.949 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-4, subscription-name4, 11] Getting connection from pool +2017-06-20 18:07:25.949 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-5, subscription-name5, 12] Getting connection from pool +2017-06-20 18:07:25.949 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-6, subscription-name6, 13] Getting connection from pool +2017-06-20 18:07:25.949 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(127)- Curl Lookup Request sent forhttp://localhost:8765/lookup/v2/destination/persistent/property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-0 +2017-06-20 18:07:25.951 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(139)- Response received for url http://localhost:8765/lookup/v2/destination/persistent/property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-0 code 200 +2017-06-20 18:07:25.951 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(207)- parseLookupData = { LookupDataResult [brokerUrl_ = pulsar://localhost:8885] [brokerUrlSsl_ = ] [partitions = 0] [authoritative = 0] [redirect = 0] +2017-06-20 18:07:25.951 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(127)- Curl Lookup Request sent forhttp://localhost:8765/lookup/v2/destination/persistent/property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-1 +2017-06-20 18:07:25.953 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(139)- Response received for url http://localhost:8765/lookup/v2/destination/persistent/property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-1 code 200 +2017-06-20 18:07:25.953 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(207)- parseLookupData = { LookupDataResult [brokerUrl_ = pulsar://localhost:8885] [brokerUrlSsl_ = ] [partitions = 0] [authoritative = 0] [redirect = 0] +2017-06-20 18:07:25.953 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(127)- Curl Lookup Request sent forhttp://localhost:8765/lookup/v2/destination/persistent/property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-2 +2017-06-20 18:07:25.955 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(139)- Response received for url http://localhost:8765/lookup/v2/destination/persistent/property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-2 code 200 +2017-06-20 18:07:25.955 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(207)- parseLookupData = { LookupDataResult [brokerUrl_ = pulsar://localhost:8885] [brokerUrlSsl_ = ] [partitions = 0] [authoritative = 0] [redirect = 0] +2017-06-20 18:07:25.955 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(155)- [persistent://property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-0, subscription-name0, 7] Created consumer on broker [127.0.0.1:50245 -> 127.0.0.1:8885] +2017-06-20 18:07:25.955 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(127)- Curl Lookup Request sent forhttp://localhost:8765/lookup/v2/destination/persistent/property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-3 +2017-06-20 18:07:25.957 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(155)- [persistent://property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-1, subscription-name1, 8] Created consumer on broker [127.0.0.1:50245 -> 127.0.0.1:8885] +2017-06-20 18:07:25.957 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(139)- Response received for url http://localhost:8765/lookup/v2/destination/persistent/property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-3 code 200 +2017-06-20 18:07:25.957 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(207)- parseLookupData = { LookupDataResult [brokerUrl_ = pulsar://localhost:8885] [brokerUrlSsl_ = ] [partitions = 0] [authoritative = 0] [redirect = 0] +2017-06-20 18:07:25.957 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(127)- Curl Lookup Request sent forhttp://localhost:8765/lookup/v2/destination/persistent/property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-4 +2017-06-20 18:07:25.959 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(139)- Response received for url http://localhost:8765/lookup/v2/destination/persistent/property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-4 code 200 +2017-06-20 18:07:25.959 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(207)- parseLookupData = { LookupDataResult [brokerUrl_ = pulsar://localhost:8885] [brokerUrlSsl_ = ] [partitions = 0] [authoritative = 0] [redirect = 0] +2017-06-20 18:07:25.959 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(155)- [persistent://property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-2, subscription-name2, 9] Created consumer on broker [127.0.0.1:50245 -> 127.0.0.1:8885] +2017-06-20 18:07:25.959 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(127)- Curl Lookup Request sent forhttp://localhost:8765/lookup/v2/destination/persistent/property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-5 +2017-06-20 18:07:25.961 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(155)- [persistent://property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-3, subscription-name3, 10] Created consumer on broker [127.0.0.1:50245 -> 127.0.0.1:8885] +2017-06-20 18:07:25.961 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(139)- Response received for url http://localhost:8765/lookup/v2/destination/persistent/property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-5 code 200 +2017-06-20 18:07:25.961 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(207)- parseLookupData = { LookupDataResult [brokerUrl_ = pulsar://localhost:8885] [brokerUrlSsl_ = ] [partitions = 0] [authoritative = 0] [redirect = 0] +2017-06-20 18:07:25.961 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(127)- Curl Lookup Request sent forhttp://localhost:8765/lookup/v2/destination/persistent/property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-6 +2017-06-20 18:07:25.963 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(155)- [persistent://property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-4, subscription-name4, 11] Created consumer on broker [127.0.0.1:50245 -> 127.0.0.1:8885] +2017-06-20 18:07:25.963 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(139)- Response received for url http://localhost:8765/lookup/v2/destination/persistent/property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-6 code 200 +2017-06-20 18:07:25.963 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(207)- parseLookupData = { LookupDataResult [brokerUrl_ = pulsar://localhost:8885] [brokerUrlSsl_ = ] [partitions = 0] [authoritative = 0] [redirect = 0] +2017-06-20 18:07:25.965 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(155)- [persistent://property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-5, subscription-name5, 12] Created consumer on broker [127.0.0.1:50245 -> 127.0.0.1:8885] +2017-06-20 18:07:25.966 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ConsumerImpl.cc(155)- [persistent://property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-6, subscription-name6, 13] Created consumer on broker [127.0.0.1:50245 -> 127.0.0.1:8885] +2017-06-20 18:07:25.967 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/PartitionedConsumerImpl.cc(222)- Successfully Subscribed to Partitioned Topic - persistent://property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244 with - 7 Partitions. +2017-06-20 18:07:25.967 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(127)- Curl Lookup Request sent forhttp://localhost:8765/admin/persistent/property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244/partitions +2017-06-20 18:07:25.968 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(139)- Response received for url http://localhost:8765/admin/persistent/property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244/partitions code 200 +2017-06-20 18:07:25.968 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(179)- parsePartitionData = { LookupDataResult [brokerUrl_ = ] [brokerUrlSsl_ = ] [partitions = 7] [authoritative = 0] [redirect = 0] +2017-06-20 18:07:25.969 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-0, ] Getting connection from pool +2017-06-20 18:07:25.969 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-1, ] Getting connection from pool +2017-06-20 18:07:25.969 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-2, ] Getting connection from pool +2017-06-20 18:07:25.969 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-3, ] Getting connection from pool +2017-06-20 18:07:25.969 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-4, ] Getting connection from pool +2017-06-20 18:07:25.969 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-5, ] Getting connection from pool +2017-06-20 18:07:25.969 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HandlerBase.cc(54)- [persistent://property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-6, ] Getting connection from pool +2017-06-20 18:07:25.969 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(127)- Curl Lookup Request sent forhttp://localhost:8765/lookup/v2/destination/persistent/property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-0 +2017-06-20 18:07:25.970 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(139)- Response received for url http://localhost:8765/lookup/v2/destination/persistent/property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-0 code 200 +2017-06-20 18:07:25.971 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(207)- parseLookupData = { LookupDataResult [brokerUrl_ = pulsar://localhost:8885] [brokerUrlSsl_ = ] [partitions = 0] [authoritative = 0] [redirect = 0] +2017-06-20 18:07:25.971 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(127)- Curl Lookup Request sent forhttp://localhost:8765/lookup/v2/destination/persistent/property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-1 +2017-06-20 18:07:25.972 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ProducerImpl.cc(118)- [persistent://property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-0, ] Created producer on broker [127.0.0.1:50245 -> 127.0.0.1:8885] +2017-06-20 18:07:25.972 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(139)- Response received for url http://localhost:8765/lookup/v2/destination/persistent/property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-1 code 200 +2017-06-20 18:07:25.972 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(207)- parseLookupData = { LookupDataResult [brokerUrl_ = pulsar://localhost:8885] [brokerUrlSsl_ = ] [partitions = 0] [authoritative = 0] [redirect = 0] +2017-06-20 18:07:25.972 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(127)- Curl Lookup Request sent forhttp://localhost:8765/lookup/v2/destination/persistent/property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-2 +2017-06-20 18:07:25.973 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ProducerImpl.cc(118)- [persistent://property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-1, ] Created producer on broker [127.0.0.1:50245 -> 127.0.0.1:8885] +2017-06-20 18:07:25.974 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(139)- Response received for url http://localhost:8765/lookup/v2/destination/persistent/property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-2 code 200 +2017-06-20 18:07:25.974 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(207)- parseLookupData = { LookupDataResult [brokerUrl_ = pulsar://localhost:8885] [brokerUrlSsl_ = ] [partitions = 0] [authoritative = 0] [redirect = 0] +2017-06-20 18:07:25.974 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(127)- Curl Lookup Request sent forhttp://localhost:8765/lookup/v2/destination/persistent/property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-3 +2017-06-20 18:07:25.975 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ProducerImpl.cc(118)- [persistent://property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-2, ] Created producer on broker [127.0.0.1:50245 -> 127.0.0.1:8885] +2017-06-20 18:07:25.976 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(139)- Response received for url http://localhost:8765/lookup/v2/destination/persistent/property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-3 code 200 +2017-06-20 18:07:25.976 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(207)- parseLookupData = { LookupDataResult [brokerUrl_ = pulsar://localhost:8885] [brokerUrlSsl_ = ] [partitions = 0] [authoritative = 0] [redirect = 0] +2017-06-20 18:07:25.976 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(127)- Curl Lookup Request sent forhttp://localhost:8765/lookup/v2/destination/persistent/property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-4 +2017-06-20 18:07:25.977 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ProducerImpl.cc(118)- [persistent://property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-3, ] Created producer on broker [127.0.0.1:50245 -> 127.0.0.1:8885] +2017-06-20 18:07:25.978 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(139)- Response received for url http://localhost:8765/lookup/v2/destination/persistent/property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-4 code 200 +2017-06-20 18:07:25.978 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(207)- parseLookupData = { LookupDataResult [brokerUrl_ = pulsar://localhost:8885] [brokerUrlSsl_ = ] [partitions = 0] [authoritative = 0] [redirect = 0] +2017-06-20 18:07:25.978 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(127)- Curl Lookup Request sent forhttp://localhost:8765/lookup/v2/destination/persistent/property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-5 +2017-06-20 18:07:25.979 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ProducerImpl.cc(118)- [persistent://property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-4, ] Created producer on broker [127.0.0.1:50245 -> 127.0.0.1:8885] +2017-06-20 18:07:25.980 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(139)- Response received for url http://localhost:8765/lookup/v2/destination/persistent/property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-5 code 200 +2017-06-20 18:07:25.980 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(207)- parseLookupData = { LookupDataResult [brokerUrl_ = pulsar://localhost:8885] [brokerUrlSsl_ = ] [partitions = 0] [authoritative = 0] [redirect = 0] +2017-06-20 18:07:25.980 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(127)- Curl Lookup Request sent forhttp://localhost:8765/lookup/v2/destination/persistent/property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-6 +2017-06-20 18:07:25.981 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ProducerImpl.cc(118)- [persistent://property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-5, ] Created producer on broker [127.0.0.1:50245 -> 127.0.0.1:8885] +2017-06-20 18:07:25.982 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(139)- Response received for url http://localhost:8765/lookup/v2/destination/persistent/property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-6 code 200 +2017-06-20 18:07:25.982 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/HTTPLookupService.cc(207)- parseLookupData = { LookupDataResult [brokerUrl_ = pulsar://localhost:8885] [brokerUrlSsl_ = ] [partitions = 0] [authoritative = 0] [redirect = 0] +2017-06-20 18:07:25.983 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/ProducerImpl.cc(118)- [persistent://property/cluster/namespace/testAsyncCallOnPartitionedTopic-1498007244-partition-6, ] Created producer on broker [127.0.0.1:50245 -> 127.0.0.1:8885] +[ OK ] ConsumerStatsTest.testAsyncCallOnPartitionedTopic (5706 ms) +[----------] 4 tests from ConsumerStatsTest (13375 ms total) + +[----------] 1 test from ConsumerTest +[ RUN ] ConsumerTest.consumerNotInitialized +[ OK ] ConsumerTest.consumerNotInitialized (0 ms) +[----------] 1 test from ConsumerTest (0 ms total) + +[----------] 9 tests from DestinationNameTest +[ RUN ] DestinationNameTest.testLookup +[ OK ] DestinationNameTest.testLookup (1 ms) +[ RUN ] DestinationNameTest.testDestinationName +[ OK ] DestinationNameTest.testDestinationName (0 ms) +[ RUN ] DestinationNameTest.testDestinationNameWithSlashes +[ OK ] DestinationNameTest.testDestinationNameWithSlashes (0 ms) +[ RUN ] DestinationNameTest.testEmptyClusterName +2017-06-20 18:07:30.606 : ERROR /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/DestinationName.cc(159)- Destination Name Validation Failed +[ OK ] DestinationNameTest.testEmptyClusterName (0 ms) +[ RUN ] DestinationNameTest.testExtraSlashes +2017-06-20 18:07:30.606 : ERROR /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/DestinationName.cc(159)- Destination Name Validation Failed +2017-06-20 18:07:30.606 : ERROR /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/DestinationName.cc(159)- Destination Name Validation Failed +[ OK ] DestinationNameTest.testExtraSlashes (0 ms) +[ RUN ] DestinationNameTest.testIllegalCharacters +2017-06-20 18:07:30.606 : ERROR /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/DestinationName.cc(159)- Destination Name Validation Failed +[ OK ] DestinationNameTest.testIllegalCharacters (0 ms) +[ RUN ] DestinationNameTest.testIllegalUrl +2017-06-20 18:07:30.606 : ERROR /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/DestinationName.cc(54)- Destination Name Invalid, domain not present - persistent:::/property/cluster/namespace/destination +2017-06-20 18:07:30.606 : ERROR /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/DestinationName.cc(153)- Destination Name Initialization failed +[ OK ] DestinationNameTest.testIllegalUrl (0 ms) +[ RUN ] DestinationNameTest.testEmptyString +2017-06-20 18:07:30.606 : ERROR /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/DestinationName.cc(54)- Destination Name Invalid, domain not present - +2017-06-20 18:07:30.606 : ERROR /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/DestinationName.cc(153)- Destination Name Initialization failed +[ OK ] DestinationNameTest.testEmptyString (0 ms) +[ RUN ] DestinationNameTest.testExtraArguments +2017-06-20 18:07:30.606 : ERROR /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/DestinationName.cc(54)- Destination Name Invalid, domain not present - persistent:::/property/cluster/namespace/destination/some/extra/args +2017-06-20 18:07:30.606 : ERROR /Users/jai1/cpp/pulsar/pulsar-client-cpp/lib/DestinationName.cc(153)- Destination Name Initialization failed +[ OK ] DestinationNameTest.testExtraArguments (0 ms) +[----------] 9 tests from DestinationNameTest (1 ms total) + +[----------] 3 tests from LatchTest +[ RUN ] LatchTest.testCountDown +2017-06-20 18:07:30.628 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/tests/LatchTest.cc(45)- Service service3 is up +2017-06-20 18:07:30.636 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/tests/LatchTest.cc(45)- Service service2 is up +2017-06-20 18:07:30.659 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/tests/LatchTest.cc(45)- Service service1 is up +[ OK ] LatchTest.testCountDown (53 ms) +[ RUN ] LatchTest.testLatchCount +2017-06-20 18:07:30.680 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/tests/LatchTest.cc(45)- Service service3 is up +2017-06-20 18:07:30.689 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/tests/LatchTest.cc(45)- Service service2 is up +2017-06-20 18:07:30.713 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/tests/LatchTest.cc(45)- Service service1 is up +[ OK ] LatchTest.testLatchCount (54 ms) +[ RUN ] LatchTest.testTimedWait +2017-06-20 18:07:30.744 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/tests/LatchTest.cc(45)- Service service2 is up +2017-06-20 18:07:30.767 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/tests/LatchTest.cc(45)- Service service3 is up +2017-06-20 18:07:30.767 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/tests/LatchTest.cc(45)- Service service1 is up +2017-06-20 18:07:30.801 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/tests/LatchTest.cc(45)- Service service5 is up +2017-06-20 18:07:30.819 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/tests/LatchTest.cc(45)- Service service6 is up +2017-06-20 18:07:30.819 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/tests/LatchTest.cc(45)- Service service4 is up +[ OK ] LatchTest.testTimedWait (106 ms) +[----------] 3 tests from LatchTest (213 ms total) + +[----------] 1 test from LoggerTest +[ RUN ] LoggerTest.testLogger +2017-06-20 18:07:30.819 : INFO /Users/jai1/cpp/pulsar/pulsar-client-cpp/tests/LoggerTest.cc(25)- Testing logger with arguments 5 +[ OK ] LoggerTest.testLogger (0 ms) +[----------] 1 test from LoggerTest (0 ms total) + +[----------] 3 tests from MessageTest +[ RUN ] MessageTest.testMessageContents +[ OK ] MessageTest.testMessageContents (0 ms) +[ RUN ] MessageTest.testAllocatedContents +[ OK ] MessageTest.testAllocatedContents (0 ms) +[ RUN ] MessageTest.testProperties +[ OK ] MessageTest.testProperties (0 ms) +[----------] 3 tests from MessageTest (0 ms total) + +[----------] 1 test from NamespaceNameTest +[ RUN ] NamespaceNameTest.testNamespaceName +[ OK ] NamespaceNameTest.testNamespaceName (0 ms) +[----------] 1 test from NamespaceNameTest (0 ms total) + +[----------] 1 test from ProducerTest +[ RUN ] ProducerTest.producerNotInitialized +[ OK ] ProducerTest.producerNotInitialized (0 ms) +[----------] 1 test from ProducerTest (0 ms total) + +[----------] 1 test from UrlTest +[ RUN ] UrlTest.testUrl +[ OK ] UrlTest.testUrl (0 ms) +[----------] 1 test from UrlTest (0 ms total) + +[----------] Global test environment tear-down +[==========] 72 tests from 16 test cases ran. (262230 ms total) +[ PASSED ] 70 tests. +[ FAILED ] 2 tests, listed below: +[ FAILED ] AuthPluginTest.testCreate +[ FAILED ] AuthPluginTest.testTls + + 2 FAILED TESTS diff --git a/pulsar-client-cpp/tests/BasicEndToEndTest.cc b/pulsar-client-cpp/tests/BasicEndToEndTest.cc index 7993ca86db8d9..8541adefbf71c 100644 --- a/pulsar-client-cpp/tests/BasicEndToEndTest.cc +++ b/pulsar-client-cpp/tests/BasicEndToEndTest.cc @@ -28,7 +28,7 @@ #include "CustomRoutingPolicy.h" #include #include - +#include "PulsarFriend.h" #include "HttpHelper.h" #include "lib/Future.h" @@ -49,14 +49,20 @@ static void messageListenerFunction(Consumer consumer, const Message& msg) { consumer.acknowledge(msg); } -static void sendCallBack(Result r, const Message& msg) { +static void sendCallBack(Result r, const Message& msg, std::string prefix) { ASSERT_EQ(r, ResultOk); - std::string prefix = "msg-batch-"; std::string messageContent = prefix + boost::lexical_cast(globalTestBatchMessagesCounter++); ASSERT_EQ(messageContent, msg.getDataAsString()); LOG_DEBUG("Received publish acknowledgement for " << msg.getDataAsString()); } +static void sendCallBack(Result r, const Message& msg, std::string prefix, double percentage, uint64_t delayInMicros) { + if ((rand() % 100) <= percentage) { + usleep(delayInMicros); + } + sendCallBack(r, msg, prefix); +} + TEST(BasicEndToEndTest, testBatchMessages) { ClientConfiguration config; @@ -68,10 +74,12 @@ TEST(BasicEndToEndTest, testBatchMessages) // Enable batching on producer side int batchSize = 2; int numOfMessages = 1000; + ProducerConfiguration conf; conf.setCompressionType(CompressionLZ4); conf.setBatchingMaxMessages(batchSize); conf.setBatchingEnabled(true); + conf.setBlockIfQueueFull(true); Promise producerPromise; client.createProducerAsync(topicName, conf, WaitForCallbackValue(producerPromise)); @@ -101,37 +109,36 @@ TEST(BasicEndToEndTest, testBatchMessages) for (int i = 0; i(i); Message msg = MessageBuilder().setContent(messageContent).setProperty("msgIndex", boost::lexical_cast(i)).build(); - producer.sendAsync(msg, &sendCallBack); - LOG_INFO("sending message " << messageContent); + producer.sendAsync(msg, boost::bind(&sendCallBack, _1, _2, prefix)); + LOG_DEBUG("sending message " << messageContent); } Message receivedMsg; int i = 0; while (consumer.receive(receivedMsg, 5000) == ResultOk) { std::string expectedMessageContent = prefix + boost::lexical_cast(i); - LOG_INFO("Received Message with [ content - " << receivedMsg.getDataAsString() << "] [ messageID = " << receivedMsg.getMessageId() << "]"); + LOG_DEBUG("Received Message with [ content - " << receivedMsg.getDataAsString() << "] [ messageID = " << receivedMsg.getMessageId() << "]"); ASSERT_EQ(receivedMsg.getProperty("msgIndex"), boost::lexical_cast(i++)); ASSERT_EQ(expectedMessageContent, receivedMsg.getDataAsString()); ASSERT_EQ(ResultOk, consumer.acknowledge(receivedMsg)); } // Number of messages produced ASSERT_EQ(globalTestBatchMessagesCounter, numOfMessages); - + globalTestBatchMessagesCounter = 0; // Number of messages consumed ASSERT_EQ(i, numOfMessages); } -void resendMessage(Result r, const Message& msg, Producer &producer) { +void resendMessage(Result r, const Message msg, Producer producer) { Lock lock(mutex_); if (r != ResultOk) { LOG_DEBUG("globalResendMessageCount" << globalResendMessageCount); - if (globalResendMessageCount++ >= 3) { + if (++globalResendMessageCount >= 3) { return; } - lock.unlock(); } - usleep(2 * 1000); + lock.unlock(); producer.sendAsync(MessageBuilder().build(), boost::bind(resendMessage, _1, _2, producer)); } @@ -274,9 +281,14 @@ TEST(BasicEndToEndTest, testLookupThrottling) { Consumer consumer; result = client.subscribe("persistent://prop/unit/ns1/my-topic-3", "my-sub-name", consumer); + + // Clean dangling subscription + consumer.unsubscribe(); + result = client.subscribe("persistent://prop/unit/ns1/my-topic-3", "my-sub-name", consumer); + ASSERT_EQ(ResultOk, result); - // Send 10 messages synchronosly + // Send 10 messages synchronously std::string msgContent = "msg-content"; LOG_INFO("Publishing 10 messages synchronously"); int numMsg = 0; @@ -290,7 +302,7 @@ TEST(BasicEndToEndTest, testLookupThrottling) { Message msgReceived; for (int i = 0; i < 10; i++) { consumer.receive(msgReceived, 1000); - LOG_INFO("Received message :" << msgReceived.getMessageId()); + LOG_DEBUG("Received message :" << msgReceived.getMessageId()); ASSERT_EQ(msgContent, msgReceived.getDataAsString()); ASSERT_EQ(boost::lexical_cast(i), msgReceived.getProperty("msgIndex")); ASSERT_EQ(ResultOk, consumer.acknowledgeCumulative(msgReceived)); @@ -397,8 +409,8 @@ TEST(BasicEndToEndTest, testPartitionedProducerConsumer) ss << nanoSeconds; Message msg = MessageBuilder().setContent(ss.str()).setPartitionKey(ss.str()).build(); ASSERT_EQ(ResultOk, producer.send(msg)); - LOG_INFO("Message Timestamp is " << msg.getPublishTimestamp()); - LOG_INFO("Message is " << msg); + LOG_DEBUG("Message Timestamp is " << msg.getPublishTimestamp()); + LOG_DEBUG("Message is " << msg); } ASSERT_EQ(consumer.getSubscriptionName(), "subscription-A"); @@ -827,11 +839,134 @@ TEST(BasicEndToEndTest, testMessageListenerPause) // Send asynchronously for 3 seconds // Expect timeouts since we have set timeout to 1 ms // On receiving timeout send the message using the CMS client IO thread via cb function. - for (int i = 0; i<1000; i++) { + for (int i = 0; i<10000; i++) { producer.sendAsync(MessageBuilder().build(), boost::bind(resendMessage, _1, _2, producer)); } // 3 seconds usleep(3 * 1000 * 1000); + producer.close(); Lock lock(mutex_); ASSERT_GE(globalResendMessageCount, 3); } + + TEST(BasicEndToEndTest, testStatsLatencies) +{ + ClientConfiguration config; + config.setIOThreads(1); + config.setMessageListenerThreads(1); + config.setStatsIntervalInSeconds(5); + Client client(lookupUrl, config); + std::string topicName = "persistent://property/cluster/namespace/testStatsLatencies"; + std::string subName = "subscription-name"; + Producer producer; + + // Start Producer and Consumer + int numOfMessages = 1000; + + Promise producerPromise; + client.createProducerAsync(topicName, WaitForCallbackValue(producerPromise)); + Future producerFuture = producerPromise.getFuture(); + Result result = producerFuture.get(producer); + ASSERT_EQ(ResultOk, result); + + Consumer consumer; + Promise consumerPromise; + client.subscribeAsync(topicName, subName, WaitForCallbackValue(consumerPromise)); + Future consumerFuture = consumerPromise.getFuture(); + result = consumerFuture.get(consumer); + ASSERT_EQ(ResultOk, result); + + // handling dangling subscriptions + consumer.unsubscribe(); + client.subscribe(topicName, subName, consumer); + + std::string temp = producer.getTopic(); + ASSERT_EQ(temp, topicName); + temp = consumer.getTopic(); + ASSERT_EQ(temp, topicName); + ASSERT_EQ(consumer.getSubscriptionName(), subName); + + ProducerStatsImplPtr producerStatsImplPtr = PulsarFriend::getProducerStatsPtr(producer); + + // Send Asynchronously + std::string prefix = "msg-stats-"; + for (int i = 0; i < numOfMessages; i++) { + std::string messageContent = prefix + boost::lexical_cast(i); + Message msg = MessageBuilder().setContent(messageContent).setProperty( + "msgIndex", boost::lexical_cast(i)).build(); + producer.sendAsync(msg, boost::bind(&sendCallBack, _1, _2, prefix, 15, 20 * 1e3)); + LOG_DEBUG("sending message " << messageContent); + } + + // Wait for all messages to be acked by broker + while (PulsarFriend::sum(producerStatsImplPtr->getTotalSendMap()) < numOfMessages) { + usleep(1000); // 1 ms + } + + // Get latencies + LatencyAccumulator totalLatencyAccumulator = + producerStatsImplPtr->getTotalLatencyAccumulator(); + boost::accumulators::detail::extractor_result::type totalLatencies = + boost::accumulators::extended_p_square(totalLatencyAccumulator); + + LatencyAccumulator latencyAccumulator = producerStatsImplPtr->getLatencyAccumulator(); + boost::accumulators::detail::extractor_result::type latencies = + boost::accumulators::extended_p_square(latencyAccumulator); + + // Since 15% of the messages have a delay of + ASSERT_EQ((uint64_t )latencies[1], (uint64_t )totalLatencies[1]); + ASSERT_EQ((uint64_t )latencies[2], (uint64_t )totalLatencies[2]); + ASSERT_EQ((uint64_t )latencies[3], (uint64_t )totalLatencies[3]); + + ASSERT_GE((uint64_t )latencies[1], 20 * 1000); + ASSERT_GE((uint64_t )latencies[2], 20 * 1000); + ASSERT_GE((uint64_t )latencies[3], 20 * 1000); + + ASSERT_GE((uint64_t )totalLatencies[1], 20 * 1000); + ASSERT_GE((uint64_t )totalLatencies[2], 20 * 1000); + ASSERT_GE((uint64_t )totalLatencies[3], 20 * 1000); + + while (producerStatsImplPtr->getNumMsgsSent() != 0) { + usleep(1e6); // wait till stats flush + } + + usleep(1 * 1e6); // 1 second + + latencyAccumulator = producerStatsImplPtr->getLatencyAccumulator(); + latencies = boost::accumulators::extended_p_square(latencyAccumulator); + + totalLatencyAccumulator = producerStatsImplPtr->getTotalLatencyAccumulator(); + totalLatencies = boost::accumulators::extended_p_square(totalLatencyAccumulator); + + ASSERT_NE((uint64_t )latencies[1], (uint64_t )totalLatencies[1]); + ASSERT_NE((uint64_t )latencies[2], (uint64_t )totalLatencies[2]); + ASSERT_NE((uint64_t )latencies[3], (uint64_t )totalLatencies[3]); + + ASSERT_EQ((uint64_t )latencies[1], 0); + ASSERT_EQ((uint64_t )latencies[2], 0); + ASSERT_EQ((uint64_t )latencies[3], 0); + + ASSERT_GE((uint64_t )totalLatencies[1], 20 * 1000); + ASSERT_GE((uint64_t )totalLatencies[2], 20 * 1000); + ASSERT_GE((uint64_t )totalLatencies[3], 20 * 1000); + + Message receivedMsg; + int i = 0; + ConsumerStatsImplPtr consumerStatsImplPtr = PulsarFriend::getConsumerStatsPtr(consumer); + + while (consumer.receive(receivedMsg, 5000) == ResultOk) { + std::string expectedMessageContent = prefix + boost::lexical_cast(i); + LOG_DEBUG( + "Received Message with [ content - " << receivedMsg.getDataAsString() << "] [ messageID = " << receivedMsg.getMessageId() << "]"); + ASSERT_EQ(receivedMsg.getProperty("msgIndex"), boost::lexical_cast(i++)); + ASSERT_EQ(PulsarFriend::sum(consumerStatsImplPtr->getTotalReceivedMsgMap()), i); + ASSERT_EQ(expectedMessageContent, receivedMsg.getDataAsString()); + ASSERT_EQ(PulsarFriend::sum(consumerStatsImplPtr->getTotalAckedMsgMap()), i - 1); + ASSERT_EQ(ResultOk, consumer.acknowledge(receivedMsg)); + ASSERT_EQ(PulsarFriend::sum(consumerStatsImplPtr->getTotalAckedMsgMap()), i); + } + // Number of messages consumed + ASSERT_EQ(i, numOfMessages); +} diff --git a/pulsar-client-cpp/tests/BatchMessageTest.cc b/pulsar-client-cpp/tests/BatchMessageTest.cc index 81e3bc1172ec1..ecdfb299bea37 100644 --- a/pulsar-client-cpp/tests/BatchMessageTest.cc +++ b/pulsar-client-cpp/tests/BatchMessageTest.cc @@ -78,7 +78,11 @@ TEST(BatchMessageTest, testProducerConfig) { TEST(BatchMessageTest, testProducerTimeout) { std::string testName=boost::lexical_cast(epochTime) + "testProducerTimeout"; - Client client(lookupUrl); + ClientConfiguration clientConf; + clientConf.setStatsIntervalInSeconds(1); + + + Client client(lookupUrl, clientConf); std::string topicName = "persistent://property/cluster/namespace/" + testName; std::string subName = "subscription-name"; Producer producer; @@ -116,29 +120,32 @@ TEST(BatchMessageTest, testProducerTimeout) { ASSERT_EQ(temp, topicName); ASSERT_EQ(consumer.getSubscriptionName(), subName); + ProducerStatsImplPtr producerStatsImplPtr = PulsarFriend::getProducerStatsPtr(producer); // Send Asynchronously std::string prefix = "msg-batch-test-produce-timeout-"; for (int i = 0; i(i); Message msg = MessageBuilder().setContent(messageContent).setProperty("type", "batch").setProperty("msgIndex", boost::lexical_cast(i)).build(); - LOG_INFO("sending message " << messageContent); + LOG_DEBUG("sending message " << messageContent); clock_t start, end; /* Start the timer */ start = time(NULL); - LOG_INFO("start = "<(i++)); ASSERT_EQ(receivedMsg.getProperty("type"), "batch"); ASSERT_EQ(expectedMessageContent, receivedMsg.getDataAsString()); @@ -190,25 +197,35 @@ TEST(BatchMessageTest, testBatchSizeInBytes) { ASSERT_EQ(temp, topicName); ASSERT_EQ(consumer.getSubscriptionName(), subName); + ProducerStatsImplPtr producerStatsImplPtr = PulsarFriend::getProducerStatsPtr(producer); // Send Asynchronously std::string prefix = "12345678"; for (int i = 0; i(i); Message msg = MessageBuilder().setContent(messageContent).setProperty("msgIndex", boost::lexical_cast(i)).build(); producer.sendAsync(msg, &sendCallBack); - LOG_INFO("sending message " << messageContent); + ASSERT_EQ(producerStatsImplPtr->getNumMsgsSent(), i+1); + ASSERT_LT(PulsarFriend::sum(producerStatsImplPtr->getSendMap()), i+1); + ASSERT_EQ(producerStatsImplPtr->getTotalMsgsSent(), i+1); + ASSERT_LT(PulsarFriend::sum(producerStatsImplPtr->getTotalSendMap()), i+1); + LOG_DEBUG("sending message " << messageContent); } Message receivedMsg; int i = 0; while (consumer.receive(receivedMsg, 5000) == ResultOk) { std::string expectedMessageContent = prefix + boost::lexical_cast(i); - LOG_INFO("Received Message with [ content - " << receivedMsg.getDataAsString() << "] [ messageID = " << receivedMsg.getMessageId() << "]"); + LOG_DEBUG("Received Message with [ content - " << receivedMsg.getDataAsString() << "] [ messageID = " << receivedMsg.getMessageId() << "]"); ASSERT_LT(pulsar::PulsarFriend::getBatchIndex((BatchMessageId&)receivedMsg.getMessageId()),2); ASSERT_EQ(receivedMsg.getProperty("msgIndex"), boost::lexical_cast(i++)); ASSERT_EQ(expectedMessageContent, receivedMsg.getDataAsString()); ASSERT_EQ(ResultOk, consumer.acknowledge(receivedMsg)); } + + // Check stats + ASSERT_EQ(PulsarFriend::sum(producerStatsImplPtr->getSendMap()),numOfMessages); + ASSERT_EQ(PulsarFriend::sum(producerStatsImplPtr->getTotalSendMap()),numOfMessages); + // Number of messages produced ASSERT_EQ(globalTestBatchMessagesCounter, numOfMessages); @@ -220,8 +237,10 @@ TEST(BatchMessageTest, testSmallReceiverQueueSize) { std::string testName=boost::lexical_cast(epochTime) + "testSmallReceiverQueueSize"; globalTestBatchMessagesCounter=0; + ClientConfiguration clientConf; + clientConf.setStatsIntervalInSeconds(20); - Client client(lookupUrl); + Client client(lookupUrl, clientConf); std::string topicName = "persistent://property/cluster/namespace/" + testName; std::string subName = "subscription-name"; Producer producer; @@ -234,6 +253,7 @@ TEST(BatchMessageTest, testSmallReceiverQueueSize) { conf.setBatchingMaxMessages(batchSize); conf.setBatchingMaxPublishDelayMs(1); conf.setBatchingEnabled(true); + conf.setMaxPendingMessages(numOfMessages + 1); Promise producerPromise; client.createProducerAsync(topicName, conf, WaitForCallbackValue(producerPromise)); @@ -261,25 +281,44 @@ TEST(BatchMessageTest, testSmallReceiverQueueSize) { ASSERT_EQ(temp, topicName); ASSERT_EQ(consumer.getSubscriptionName(), subName); + ProducerStatsImplPtr producerStatsImplPtr = PulsarFriend::getProducerStatsPtr(producer); // Send Asynchronously std::string prefix = testName; for (int i = 0; i(i); Message msg = MessageBuilder().setContent(messageContent).setProperty("msgIndex", boost::lexical_cast(i)).build(); producer.sendAsync(msg, &sendCallBack); + ASSERT_EQ(producerStatsImplPtr->getTotalMsgsSent(), i+1); + ASSERT_LE(PulsarFriend::sum(producerStatsImplPtr->getTotalSendMap()), i+1); LOG_DEBUG("sending message " << messageContent); } - usleep(10 * 1000 * 1000); Message receivedMsg; int i = 0; - while (consumer.receive(receivedMsg, 10000) == ResultOk) { + for (i = 0; i < numOfMessages; i++) { + consumer.receive(receivedMsg); std::string expectedMessageContent = prefix + boost::lexical_cast(i); LOG_DEBUG("Received Message with [ content - " << receivedMsg.getDataAsString() << "] [ messageID = " << receivedMsg.getMessageId() << "]"); - ASSERT_EQ(receivedMsg.getProperty("msgIndex"), boost::lexical_cast(i++)); + ASSERT_EQ(receivedMsg.getProperty("msgIndex"), boost::lexical_cast(i)); ASSERT_EQ(expectedMessageContent, receivedMsg.getDataAsString()); - ASSERT_EQ(ResultOk, consumer.acknowledge(receivedMsg)); + ASSERT_EQ(ResultOk, consumer.acknowledge(receivedMsg)); } + + ConsumerStatsImplPtr consumerStatsImplPtr = PulsarFriend::getConsumerStatsPtr(consumer); + unsigned long t = consumerStatsImplPtr->getAckedMsgMap().at(std::make_pair(ResultOk, proto::CommandAck_AckType_Individual)); + ASSERT_EQ(t, numOfMessages); + ASSERT_EQ(PulsarFriend::sum(consumerStatsImplPtr->getAckedMsgMap()), numOfMessages); + ASSERT_EQ(PulsarFriend::sum(consumerStatsImplPtr->getTotalAckedMsgMap()), numOfMessages); + ASSERT_EQ(PulsarFriend::sum(consumerStatsImplPtr->getReceivedMsgMap()), numOfMessages); + ASSERT_EQ(PulsarFriend::sum(consumerStatsImplPtr->getTotalReceivedMsgMap()), numOfMessages); + ASSERT_EQ(consumerStatsImplPtr->getTotalNumBytesRecieved(), consumerStatsImplPtr->getNumBytesRecieved()); + usleep(20 * 1e6); + ASSERT_NE(consumerStatsImplPtr->getTotalNumBytesRecieved(), consumerStatsImplPtr->getNumBytesRecieved()); + ASSERT_EQ(PulsarFriend::sum(consumerStatsImplPtr->getTotalAckedMsgMap()), numOfMessages); + ASSERT_EQ(PulsarFriend::sum(consumerStatsImplPtr->getTotalReceivedMsgMap()), numOfMessages); + ASSERT_EQ(PulsarFriend::sum(consumerStatsImplPtr->getAckedMsgMap()), 0); + ASSERT_EQ(PulsarFriend::sum(consumerStatsImplPtr->getReceivedMsgMap()), 0); + // Number of messages produced ASSERT_EQ(globalTestBatchMessagesCounter, numOfMessages); @@ -290,8 +329,10 @@ TEST(BatchMessageTest, testSmallReceiverQueueSize) { TEST(BatchMessageTest, testIndividualAck) { std::string testName = boost::lexical_cast(epochTime) + "testIndividualAck"; + ClientConfiguration clientConfig; + clientConfig.setStatsIntervalInSeconds(1); - Client client(lookupUrl); + Client client(lookupUrl, clientConfig); std::string topicName = "persistent://property/cluster/namespace/" + testName; std::string subName = "subscription-name"; Producer producer; @@ -338,14 +379,14 @@ TEST(BatchMessageTest, testIndividualAck) { Message msg = MessageBuilder().setContent(messageContent).setProperty( "msgIndex", boost::lexical_cast < std::string > (i)).build(); producer.sendAsync(msg, &sendCallBack); - LOG_INFO("sending message " << messageContent); + LOG_DEBUG("sending message " << messageContent); } globalTestBatchMessagesCounter = 0; Message receivedMsg; int i = 0; while (consumer.receive(receivedMsg, 5000) == ResultOk) { std::string expectedMessageContent = prefix + boost::lexical_cast < std::string > (i); - LOG_INFO( + LOG_DEBUG( "Received Message with [ content - " << receivedMsg.getDataAsString() << "] [ messageID = " << receivedMsg.getMessageId() << "]"); ASSERT_EQ(receivedMsg.getProperty("msgIndex"), boost::lexical_cast < std::string > (i++)); ASSERT_EQ(expectedMessageContent, receivedMsg.getDataAsString()); @@ -369,7 +410,7 @@ TEST(BatchMessageTest, testIndividualAck) { i = 0; while (consumer.receive(receivedMsg, 5000) == ResultOk) { std::string expectedMessageContent = prefix + boost::lexical_cast < std::string > (i); - LOG_INFO( + LOG_DEBUG( "Received Message with [ content - " << receivedMsg.getDataAsString() << "] [ messageID = " << receivedMsg.getMessageId() << "]"); ASSERT_EQ(receivedMsg.getProperty("msgIndex"), boost::lexical_cast < std::string > (i++)); ASSERT_EQ(expectedMessageContent, receivedMsg.getDataAsString()); @@ -391,7 +432,7 @@ TEST(BatchMessageTest, testIndividualAck) { i = 0; while (consumer.receive(receivedMsg, 5000) == ResultOk) { std::string expectedMessageContent = prefix + boost::lexical_cast < std::string > (i + numOfMessages/2); - LOG_INFO( + LOG_DEBUG( "Received Message with [ content - " << receivedMsg.getDataAsString() << "] [ messageID = " << receivedMsg.getMessageId() << "]"); ASSERT_EQ(receivedMsg.getProperty("msgIndex"), boost::lexical_cast < std::string > (i++ + numOfMessages/2)); ASSERT_EQ(expectedMessageContent, receivedMsg.getDataAsString()); @@ -413,7 +454,7 @@ TEST(BatchMessageTest, testIndividualAck) { i = 0; while (consumer.receive(receivedMsg, 5000) == ResultOk) { std::string expectedMessageContent = prefix + boost::lexical_cast < std::string > (i + numOfMessages/2); - LOG_INFO( + LOG_DEBUG( "Received Message with [ content - " << receivedMsg.getDataAsString() << "] [ messageID = " << receivedMsg.getMessageId() << "]"); ASSERT_EQ(receivedMsg.getProperty("msgIndex"), boost::lexical_cast < std::string > (i++ + numOfMessages/2)); ASSERT_EQ(expectedMessageContent, receivedMsg.getDataAsString()); @@ -437,8 +478,10 @@ TEST(BatchMessageTest, testIndividualAck) { TEST(BatchMessageTest, testCumulativeAck) { std::string testName = boost::lexical_cast(epochTime) + "testCumulativeAck"; + ClientConfiguration clientConfig; + clientConfig.setStatsIntervalInSeconds(100); - Client client(lookupUrl); + Client client(lookupUrl, clientConfig); std::string topicName = "persistent://property/cluster/namespace/" + testName; std::string subName = "subscription-name"; Producer producer; @@ -479,6 +522,7 @@ TEST(BatchMessageTest, testCumulativeAck) { temp = consumer.getTopic(); ASSERT_EQ(temp, topicName); ASSERT_EQ(consumer.getSubscriptionName(), subName); + ProducerStatsImplPtr producerStatsImplPtr = PulsarFriend::getProducerStatsPtr(producer); // Send Asynchronously std::string prefix = testName; @@ -487,14 +531,15 @@ TEST(BatchMessageTest, testCumulativeAck) { Message msg = MessageBuilder().setContent(messageContent).setProperty( "msgIndex", boost::lexical_cast < std::string > (i)).build(); producer.sendAsync(msg, &sendCallBack); - LOG_INFO("sending message " << messageContent); + LOG_DEBUG("sending message " << messageContent); } Message receivedMsg; int i = 0; + ConsumerStatsImplPtr consumerStatsImplPtr = PulsarFriend::getConsumerStatsPtr(consumer); while (consumer.receive(receivedMsg, 5000) == ResultOk) { std::string expectedMessageContent = prefix + boost::lexical_cast < std::string > (i); - LOG_INFO( + LOG_DEBUG( "Received Message with [ content - " << receivedMsg.getDataAsString() << "] [ messageID = " << receivedMsg.getMessageId() << "]"); ASSERT_EQ(receivedMsg.getProperty("msgIndex"), boost::lexical_cast < std::string > (i++)); ASSERT_EQ(expectedMessageContent, receivedMsg.getDataAsString()); @@ -503,6 +548,16 @@ TEST(BatchMessageTest, testCumulativeAck) { ASSERT_EQ(ResultOk, consumer.acknowledgeCumulative(receivedMsg)); } } + + ASSERT_EQ(PulsarFriend::sum(consumerStatsImplPtr->getReceivedMsgMap()), i + 1); + // Since last receive call times out + ASSERT_EQ(consumerStatsImplPtr->getReceivedMsgMap().at(ResultOk), i); + ASSERT_EQ(consumerStatsImplPtr->getReceivedMsgMap().at(ResultTimeout), 1); + ASSERT_EQ(PulsarFriend::sum(consumerStatsImplPtr->getAckedMsgMap()), 1); + ASSERT_EQ(producerStatsImplPtr->getNumBytesSent(), consumerStatsImplPtr->getNumBytesRecieved()); + unsigned long t = consumerStatsImplPtr->getAckedMsgMap().at(std::make_pair(ResultOk, proto::CommandAck_AckType_Cumulative)); + ASSERT_EQ(t, 1); + // Number of messages produced ASSERT_EQ(globalTestBatchMessagesCounter, numOfMessages); @@ -515,10 +570,11 @@ TEST(BatchMessageTest, testCumulativeAck) { consumer.close(); client.subscribe(topicName, subName, consumerConfig, consumer); + consumerStatsImplPtr = PulsarFriend::getConsumerStatsPtr(consumer); i = 0; while (consumer.receive(receivedMsg, 5000) == ResultOk) { std::string expectedMessageContent = prefix + boost::lexical_cast < std::string > (i + 5); - LOG_INFO( + LOG_DEBUG( "Received Message with [ content - " << receivedMsg.getDataAsString() << "] [ messageID = " << receivedMsg.getMessageId() << "]"); ASSERT_EQ(receivedMsg.getProperty("msgIndex"), boost::lexical_cast < std::string > (i++ + 5)); ASSERT_EQ(expectedMessageContent, receivedMsg.getDataAsString()); @@ -528,6 +584,10 @@ TEST(BatchMessageTest, testCumulativeAck) { } } + ASSERT_EQ(PulsarFriend::sum(consumerStatsImplPtr->getAckedMsgMap()), 1); + t = consumerStatsImplPtr->getAckedMsgMap().at(std::make_pair(ResultOk, proto::CommandAck_AckType_Cumulative)); + ASSERT_EQ(t, 1); + // Number of messages consumed ASSERT_EQ(i, 10); @@ -592,14 +652,14 @@ TEST(BatchMessageTest, testMixedAck) { Message msg = MessageBuilder().setContent(messageContent).setProperty( "msgIndex", boost::lexical_cast < std::string > (i)).build(); producer.sendAsync(msg, &sendCallBack); - LOG_INFO("sending message " << messageContent); + LOG_DEBUG("sending message " << messageContent); } Message receivedMsg; int i = 0; while (consumer.receive(receivedMsg, 5000) == ResultOk) { std::string expectedMessageContent = prefix + boost::lexical_cast < std::string > (i); - LOG_INFO( + LOG_DEBUG( "Received Message with [ content - " << receivedMsg.getDataAsString() << "] [ messageID = " << receivedMsg.getMessageId() << "]"); ASSERT_EQ(receivedMsg.getProperty("msgIndex"), boost::lexical_cast < std::string > (i++)); ASSERT_EQ(expectedMessageContent, receivedMsg.getDataAsString()); @@ -623,7 +683,7 @@ TEST(BatchMessageTest, testMixedAck) { i = 0; while (consumer.receive(receivedMsg, 5000) == ResultOk) { std::string expectedMessageContent = prefix + boost::lexical_cast < std::string > (i + 10); - LOG_INFO( + LOG_DEBUG( "Received Message with [ content - " << receivedMsg.getDataAsString() << "] [ messageID = " << receivedMsg.getMessageId() << "]"); ASSERT_EQ(receivedMsg.getProperty("msgIndex"), boost::lexical_cast < std::string > (i++ + 10)); ASSERT_EQ(expectedMessageContent, receivedMsg.getDataAsString()); @@ -703,7 +763,7 @@ TEST(BatchMessageTest, testPermits) { Message msg = MessageBuilder().setContent(messageContent).setProperty( "msgIndex", boost::lexical_cast < std::string > (i)).build(); producer.sendAsync(msg, &sendCallBack); - LOG_INFO("sending message " << messageContent); + LOG_DEBUG("sending message " << messageContent); } usleep(5 * 1000 * 1000); @@ -712,7 +772,7 @@ TEST(BatchMessageTest, testPermits) { int i = 0; while (consumer.receive(receivedMsg, 5000) == ResultOk) { std::string expectedMessageContent = prefix + boost::lexical_cast < std::string > (i); - LOG_INFO( + LOG_DEBUG( "Received Message with [ content - " << receivedMsg.getDataAsString() << "] [ messageID = " << receivedMsg.getMessageId() << "]"); ASSERT_EQ(receivedMsg.getProperty("msgIndex"), boost::lexical_cast < std::string > (i++)); ASSERT_EQ(expectedMessageContent, receivedMsg.getDataAsString()); @@ -738,7 +798,7 @@ TEST(BatchMessageTest, testPermits) { Message msg = MessageBuilder().setContent(messageContent).setProperty( "msgIndex", boost::lexical_cast < std::string > (i)).build(); producer.sendAsync(msg, &sendCallBack); - LOG_INFO("sending message " << messageContent); + LOG_DEBUG("sending message " << messageContent); } usleep(5 * 1000 * 1000); @@ -748,7 +808,7 @@ TEST(BatchMessageTest, testPermits) { i = 0; while (consumer.receive(receivedMsg, 5000) == ResultOk) { std::string expectedMessageContent = prefix + boost::lexical_cast < std::string > (i); - LOG_INFO( + LOG_DEBUG( "Received Message with [ content - " << receivedMsg.getDataAsString() << "] [ messageID = " << receivedMsg.getMessageId() << "]"); ASSERT_EQ(receivedMsg.getProperty("msgIndex"), boost::lexical_cast < std::string > (i++)); ASSERT_EQ(expectedMessageContent, receivedMsg.getDataAsString()); @@ -770,7 +830,7 @@ TEST(BatchMessageTest, testPartitionedTopics) { + boost::lexical_cast(epochTime) + "/partitions"; int res = makePutRequest(url, "7"); - LOG_INFO("res = "<(i)).build(); producer.sendAsync(msg, &sendCallBackExpectingErrors); - LOG_INFO("sending message " << messageContent); + LOG_DEBUG("sending message " << messageContent); } Message receivedMsg; int i = 0; while (consumer.receive(receivedMsg, 30000) == ResultOk) { - LOG_INFO("Received Message with [ content - " << receivedMsg.getDataAsString() << "] [ messageID = " << receivedMsg.getMessageId() << "]"); + LOG_DEBUG("Received Message with [ content - " << receivedMsg.getDataAsString() << "] [ messageID = " << receivedMsg.getMessageId() << "]"); ASSERT_EQ(ResultOk, consumer.acknowledge(receivedMsg)); i++; } @@ -837,5 +897,4 @@ TEST(BatchMessageTest, testPartitionedTopics) { // Number of messages consumed ASSERT_EQ(i, numOfMessages - globalPublishCountQueueFull); - } diff --git a/pulsar-client-cpp/tests/PulsarFriend.h b/pulsar-client-cpp/tests/PulsarFriend.h index 3f87b07f81edf..fbfd9c1c4bdd7 100644 --- a/pulsar-client-cpp/tests/PulsarFriend.h +++ b/pulsar-client-cpp/tests/PulsarFriend.h @@ -17,6 +17,8 @@ * under the License. */ #include +#include +#include #include using std::string; @@ -27,5 +29,24 @@ class PulsarFriend { static int getBatchIndex(const BatchMessageId& mId) { return mId.batchIndex_; } + + static ProducerStatsImplPtr getProducerStatsPtr(Producer producer) { + ProducerImpl* producerImpl = static_cast(producer.impl_.get()); + return boost::static_pointer_cast(producerImpl->producerStatsBasePtr_); + } + + template + static unsigned long sum(std::map m) { + unsigned long sum = 0; + for (typename std::map::iterator iter = m.begin(); iter != m.end(); iter++) { + sum += iter->second; + } + return sum; + } + + static ConsumerStatsImplPtr getConsumerStatsPtr(Consumer consumer) { + ConsumerImpl* consumerImpl = static_cast(consumer.impl_.get()); + return boost::static_pointer_cast(consumerImpl->consumerStatsBasePtr_); + } }; }