Skip to content
Merged
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
12 changes: 7 additions & 5 deletions lib/PartitionedProducerImpl.cc
Original file line number Diff line number Diff line change
Expand Up @@ -92,10 +92,12 @@ unsigned int PartitionedProducerImpl::getNumPartitionsWithLock() const {
return getNumPartitions();
}

ProducerImplPtr PartitionedProducerImpl::newInternalProducer(unsigned int partition, bool lazy) {
ProducerImplPtr PartitionedProducerImpl::newInternalProducer(unsigned int partition, bool lazy,
bool retryOnCreationError) {
using namespace std::placeholders;
auto client = client_.lock();
auto producer = std::make_shared<ProducerImpl>(client, *topicName_, conf_, interceptors_, partition);
auto producer = std::make_shared<ProducerImpl>(client, *topicName_, conf_, interceptors_, partition,
retryOnCreationError);
if (!client) {
return producer;
}
Expand Down Expand Up @@ -127,13 +129,13 @@ void PartitionedProducerImpl::start() {

for (unsigned int i = 0; i < getNumPartitions(); i++) {
bool lazy = (short)i != partition;
producers_.push_back(newInternalProducer(i, lazy));
producers_.push_back(newInternalProducer(i, lazy, false));
}

producers_[partition]->start();
} else {
for (unsigned int i = 0; i < getNumPartitions(); i++) {
producers_.push_back(newInternalProducer(i, false));
producers_.push_back(newInternalProducer(i, false, false));
}

for (ProducerList::const_iterator prod = producers_.begin(); prod != producers_.end(); prod++) {
Expand Down Expand Up @@ -461,7 +463,7 @@ void PartitionedProducerImpl::handleGetPartitions(Result result,
for (unsigned int i = currentNumPartitions; i < newNumPartitions; i++) {
ProducerImplPtr producer;
try {
producer = newInternalProducer(i, lazy);
producer = newInternalProducer(i, lazy, true);
} catch (const std::runtime_error& e) {
LOG_ERROR("Failed to create producer for partition " << i << ": " << e.what());
producers.clear();
Expand Down
2 changes: 1 addition & 1 deletion lib/PartitionedProducerImpl.h
Original file line number Diff line number Diff line change
Expand Up @@ -135,7 +135,7 @@ class PartitionedProducerImpl : public ProducerImplBase,

unsigned int getNumPartitions() const;
unsigned int getNumPartitionsWithLock() const;
ProducerImplPtr newInternalProducer(unsigned int partition, bool lazy);
ProducerImplPtr newInternalProducer(unsigned int partition, bool lazy, bool retryOnCreationError);
MessageRoutingPolicyPtr getMessageRouter();
void runPartitionUpdateTask();
void getPartitionMetadata();
Expand Down
7 changes: 4 additions & 3 deletions lib/ProducerImpl.cc
Original file line number Diff line number Diff line change
Expand Up @@ -48,7 +48,7 @@ DECLARE_LOG_OBJECT()

ProducerImpl::ProducerImpl(ClientImplPtr client, const TopicName& topicName,
const ProducerConfiguration& conf, const ProducerInterceptorsPtr& interceptors,
int32_t partition)
int32_t partition, bool retryOnCreationError)
: HandlerBase(client, (partition < 0) ? topicName.toString() : topicName.getTopicPartitionName(partition),
Backoff(milliseconds(client->getClientConfig().getInitialBackoffIntervalMs()),
milliseconds(client->getClientConfig().getMaxBackoffIntervalMs()),
Expand All @@ -67,7 +67,8 @@ ProducerImpl::ProducerImpl(ClientImplPtr client, const TopicName& topicName,
dataKeyRefreshTask_(*executor_, 4 * 60 * 60 * 1000),
memoryLimitController_(client->getMemoryLimitController()),
chunkingEnabled_(conf_.isChunkingEnabled() && topicName.isPersistent() && !conf_.getBatchingEnabled()),
interceptors_(interceptors) {
interceptors_(interceptors),
retryOnCreationError_(retryOnCreationError) {
LOG_DEBUG("ProducerName - " << producerName_ << " Created producer on topic " << topic()
<< " id: " << producerId_);
if (!producerName_.empty()) {
Expand Down Expand Up @@ -273,7 +274,7 @@ Result ProducerImpl::handleCreateProducer(const ClientConnectionPtr& cnx, Result
lock.unlock();
producerCreatedPromise_.setFailed(result);
handleResult = result;
} else if (producerCreatedPromise_.isComplete()) {
} else if (producerCreatedPromise_.isComplete() || retryOnCreationError_) {
if (result == ResultProducerBlockedQuotaExceededException) {
LOG_WARN(getName() << "Backlog is exceeded on topic. Sending exception to producer");
failPendingMessages(ResultProducerBlockedQuotaExceededException, false);
Expand Down
5 changes: 4 additions & 1 deletion lib/ProducerImpl.h
Original file line number Diff line number Diff line change
Expand Up @@ -66,7 +66,8 @@ class ProducerImpl : public HandlerBase, public ProducerImplBase {
public:
ProducerImpl(ClientImplPtr client, const TopicName& topic,
const ProducerConfiguration& producerConfiguration,
const ProducerInterceptorsPtr& interceptors, int32_t partition = -1);
const ProducerInterceptorsPtr& interceptors, int32_t partition = -1,
bool retryOnCreationError = false);
~ProducerImpl();

// overrided methods from ProducerImplBase
Expand Down Expand Up @@ -202,6 +203,8 @@ class ProducerImpl : public HandlerBase, public ProducerImplBase {
boost::optional<uint64_t> topicEpoch;

ProducerInterceptorsPtr interceptors_;

bool retryOnCreationError_;
};

struct ProducerImplCmp {
Expand Down
33 changes: 33 additions & 0 deletions tests/ProducerTest.cc
Original file line number Diff line number Diff line change
Expand Up @@ -649,4 +649,37 @@ TEST(ProducerTest, testReconnectMultiConnectionsPerBroker) {
client.close();
}

TEST(ProducerTest, testFailedToCreateNewPartitionProducer) {
const std::string topic =
"public/default/testFailedToCreateNewPartitionProducer" + std::to_string(time(nullptr));
std::string topicOperateUrl = adminUrl + "admin/v2/persistent/" + topic + "/partitions";

int res = makePutRequest(topicOperateUrl, "2");
ASSERT_TRUE(res == 204 || res == 409) << "res: " << res;

ClientConfiguration clientConf;
clientConf.setPartititionsUpdateInterval(1);
Client client(serviceUrl, clientConf);
ProducerConfiguration conf;
Producer producer;
client.createProducer(topic, conf, producer);
waitUntil(std::chrono::seconds(1), [&producer]() -> bool { return producer.isConnected(); });
ASSERT_TRUE(producer.isConnected());
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

You can check the returned value of waitUntil.

ASSERT_TRUE(waitUntil(std::chrono::seconds(1), [&producer]() -> bool { return producer.isConnected(); }));

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Addressed.


PartitionedProducerImpl& partitionedProducer = PulsarFriend::getPartitionedProducerImpl(producer);
PulsarFriend::updatePartitions(partitionedProducer, 3);
std::this_thread::sleep_for(std::chrono::milliseconds(500));
auto& new_producer = PulsarFriend::getInternalProducerImpl(producer, 2);
ASSERT_FALSE(new_producer.isConnected()); // should fail with topic not found

res = makePostRequest(topicOperateUrl, "3");
ASSERT_TRUE(res == 204 || res == 409) << "res: " << res;

waitUntil(std::chrono::seconds(5), [&new_producer]() -> bool { return new_producer.isConnected(); });
ASSERT_TRUE(new_producer.isConnected());

producer.close();
client.close();
}

INSTANTIATE_TEST_CASE_P(Pulsar, ProducerTest, ::testing::Values(true, false));