Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[PIP 79][client] Add lazy-loading feature to PartitionedProducer #10279

Merged
merged 10 commits into from
Oct 15, 2021

Conversation

equanz
Copy link
Contributor

@equanz equanz commented Apr 20, 2021

Master Issue: https://github.com/apache/pulsar/wiki/PIP-79%3A-Reduce-redundant-producers-from-partitioned-producer

Motivation

Please see the PIP document.
This is a part of implementations. I will submit the next PR about PartitionedTopicStats later. #10534 is the next PR.

Modifications

  • Partitioned producer connects to partitions lazily
  • Add round-robin with limit number of partitions routing mode

Verifying this change

  • Make sure that the change passes the CI checks.

This change added tests and can be verified as follows:

  • Added test for lazy-loading
  • Added test for routing mode class

Does this pull request potentially affect one of the following parts:

If yes was chosen, please highlight the changes

  • Dependencies (does it add or upgrade a dependency): (no)
  • The public API: (no)
  • The schema: (no)
  • The default values of configurations: (no)
  • The wire protocol: (no)
  • The rest endpoints: (no)
  • The admin cli options: (no)
  • Anything that affects deployment: (no)

Documentation

  • Does this pull request introduce a new feature? (yes)
  • If yes, how is the feature documented? (not documented)

@equanz
Copy link
Contributor Author

equanz commented Apr 20, 2021

/pulsarbot run-failure-checks

@nkurihar nkurihar added area/client type/PIP type/enhancement The enhancements for the existing features or docs. e.g. reduce memory usage of the delayed messages type/feature The PR added a new feature or issue requested a new feature labels Apr 20, 2021
Copy link
Contributor

@eolivelli eolivelli left a comment

Choose a reason for hiding this comment

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

Thanks this awesome proposal.
I left some early feedback after doing a first pass.
I am not sure about concurrent aspects during producer creation. I will take another look soon

@@ -1521,6 +1522,17 @@ public void testMaxTopicsPerNamespace() throws Exception {
}

// check producer/consumer auto create partitioned topic
final Function<Producer<byte[]>, Producer<byte[]>> send = (p) -> {
Copy link
Contributor

Choose a reason for hiding this comment

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

Why are you modifying an existing test?
It is okay to refactor and reuse an existing test but in this case probably you are altering the behaviour of the existing guest and also you are not sure that you are testing your feature properly

Copy link
Contributor Author

@equanz equanz Apr 27, 2021

Choose a reason for hiding this comment

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

Why are you modifying an existing test?

Because this feature is modifying existing behavior. Currently, a partitioned producer connects to all partitions. Some cases of tests suppose "partitioned producer connects to all partitions(and also create all partition automatically)".
In this change, I modified this procedure to lazy-loading and modified test cases to follow it(here is a part of test codes for this feature).

If we should not change existing behavior, I'll change this feature to be able to stop.

log.error("[{}] Could not create internal producer. partitionIndex: {}", topic, partition,
createException);
try {
producers.remove(partition).close();
Copy link
Contributor

Choose a reason for hiding this comment

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

We are not sure that we are removing the newly create producer here.

Copy link
Contributor Author

@equanz equanz Apr 27, 2021

Choose a reason for hiding this comment

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

The reason for it is to recreate internal producer when producer creation is failed. This behavior is tested here.

Copy link
Contributor

Choose a reason for hiding this comment

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

I mean that we should test the result for "producers.remove(partition)" for equality ('==') with prod

@nkurihar
Copy link
Contributor

@sijie @eolivelli @merlimat
Could you please review this PR?

Though there are conflicts, we want to confirm the motivation and solution make sense before resolving conflicts.

@Vanlightly
Copy link
Contributor

I have submitted a PR for lazy producers in the C++ client (#11570), I did the work before seeing this PIP-79. The approach I took was to create the producers without starting the lookup and connection procedure. So the collection of producers is as big as the number of partitions, and changes when the number partitions change, but the lookup and connect procedure is only started on a producer's first message. It does not block in sendAsync but kicks off the procedure asynchronously allowing messages to be buffered until the producer is connected. In the C++ client, deadlocks were an issue so I avoided any blocking code.

I don't know if that approach makes sense for the Java client as I have not studied how the Java client works in detail yet. But that seems to be the main difference between my C++ changes and this Java client change.

codelipenghui pushed a commit that referenced this pull request Aug 16, 2021
Fixes #11496 also matches part of PIP 79.

C++ implementation that closely matches the proposed Java client changes from reducing partitioned producer connections and lookups: PR 10279

### Motivation

Producers that send messages to partitioned topics start a producer per partition, even when using single partition routing. For topics that have the combination of a large number of producers and a large number of partitions, this can put strain on the brokers. With say 1000 partitions and single partition routing with non-keyed messages, 999 topic owner lookups and producer registrations are performed that could be avoided.

PIP 79 also describes this. I wrote this before realising that PIP 79 also covers this. This implementation can be reviewed and contrasted to the Java client implementation in #10279.

### Modifications

Allows partitioned producers to start producers for individual partitions lazily. Starting a producer involves a topic owner
lookup to find out which broker is the owner of the partition, then registering the producer for that partition with the owner
broker. For topics with many partitions and when using SinglePartition routing without keyed messages, all of these
lookups and producer registrations are a waste except for the single chosen partition.

This change allows the user to control whether a producer on a partitioned topic uses this lazy start or not, via a new config
in ProducerConfiguration. When ProducerConfiguration.setLazyStartPartitionedProducers(true) is set, the PartitionedProducerImpl.start() becomes a synchronous operation that only does housekeeping (no network operations).
The producer of any given partition is started (which includes a topic owner lookup and registration) upon sending the first message to that partition. While the producer starts, messages are buffered.

The sendTimeout timer is only activated once a producer has been fully started, which should give enough time for any buffered messages to be sent. For very short send timeouts, this setting could cause send timeouts during the start phase. The default of 30s should however not cause this issue.
Comment on lines -116 to +134
return producers.get(0).getProducerName();
return producers.get(firstPartitionIndex).getProducerName();
Copy link
Contributor Author

Choose a reason for hiding this comment

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

This change breaks existing behavior. If it is not approved, I'll be changed to create the partition-0 producer for backward compatibility.

Copy link
Contributor

Choose a reason for hiding this comment

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

shouldn't be the same producerName for all the producers ?
why not computing the name in the constructor and cache it ?

Copy link
Contributor Author

@equanz equanz Sep 2, 2021

Choose a reason for hiding this comment

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

shouldn't be the same producerName for all the producers ?

In my understanding, currently producerName is calculated for each partition if doesn't set it at ProducerBuilder.

final String producerName = cmdProducer.hasProducerName() ? cmdProducer.getProducerName()
: service.generateUniqueProducerName();

if (this.producerName == null) {
this.producerName = producerName;
}

why not computing the name in the constructor and cache it ?

If needed, I'll add the behavior that compute and cache producerName to below.

producer.producerCreatedFuture().handle((prod, createException) -> {
if (createException != null) {
setState(State.Failed);
createFail.compareAndSet(null, createException);
}
// we mark success if all the partitions are created
// successfully, else we throw an exception
// due to any
// failure in one of the partitions and close the successfully
// created partitions
if (completed.incrementAndGet() == topicMetadata.numPartitions()) {
if (createFail.get() == null) {
setState(State.Ready);
log.info("[{}] Created partitioned producer", topic);
producerCreatedFuture().complete(PartitionedProducerImpl.this);
} else {
log.error("[{}] Could not create partitioned producer.", topic, createFail.get().getCause());
closeAsync().handle((ok, closeException) -> {
producerCreatedFuture().completeExceptionally(createFail.get());
client.cleanupProducer(this);
return null;
});
}
}
return null;
});

@equanz
Copy link
Contributor Author

equanz commented Aug 30, 2021

/pulsarbot run-failure-checks

@equanz
Copy link
Contributor Author

equanz commented Aug 30, 2021

Rebased to current master commit
PTAL

@equanz
Copy link
Contributor Author

equanz commented Sep 1, 2021

@eolivelli @Vanlightly
Could you please review it?

Copy link
Contributor

@eolivelli eolivelli left a comment

Choose a reason for hiding this comment

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

I left another round of comments

@merlimat @codelipenghui @rdhabalia can you please take a look ?

for (int i = 0; i < 3; i++) {
try {
producer.newMessage().value("msg".getBytes()).send();
} catch (Throwable e) {
Copy link
Contributor

Choose a reason for hiding this comment

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

the test will fail anyway, no need to catch the exception

p.newMessage().value("msg".getBytes()).send();
} catch (Throwable e) {
log.info("Exception: ", e);
fail();
Copy link
Contributor

Choose a reason for hiding this comment

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

what about

catch (Exception e) {
            log.info("Exception: ", e);
            throw new RuntimeException(e);
}

for (int i = 0; i < 2; i++) {
try {
p.newMessage().value("msg".getBytes()).send();
} catch (Throwable e) {
Copy link
Contributor

Choose a reason for hiding this comment

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

same here

.newProducer(Schema.JSON(Schemas.BytesRecord.class))
.topic(topic)
.enableBatching(false)
Copy link
Contributor

Choose a reason for hiding this comment

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

why do you need to change this test ?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

In this PR, I'll introduce internal producer lazy-loading feature. When internal producer is elected by MessageRouter and isn't created yet, create it lazily.
https://github.com/equanz/pulsar/blob/844cca7cf84b76b7d27f12de4dbf36e23cbf29e9/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PartitionedProducerImpl.java#L193-L211
Therefore, if it isn't elected by MessageRouter, doesn't create producer and also topic in this test case.

For above reason, we should add Producer#send.

When enableBatching, not easier to elect all of internal producers at client side.

if (isBatchingEnabled) { // if batching is enabled, choose partition on `partitionSwitchMs` boundary.
long currentMs = clock.millis();
return signSafeMod(currentMs / partitionSwitchMs + startPtnIdx, topicMetadata.numPartitions());

@@ -418,7 +418,11 @@ public void testSeekTimeOnPartitionedTopic() throws Exception {
long resetTimeInMillis = TimeUnit.SECONDS
.toMillis(RelativeTimeUtil.parseRelativeTimeInSeconds(resetTimeStr));
admin.topics().createPartitionedTopic(topicName, partitions);
Producer<byte[]> producer = pulsarClient.newProducer().topic(topicName).create();
Producer<byte[]> producer = pulsarClient.newProducer().topic(topicName).enableBatching(false).create();
Copy link
Contributor

Choose a reason for hiding this comment

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

why do you need to change this test ? (enableBatching(false))

@@ -863,7 +863,7 @@ public void testConsumerDistributionInFailoverSubscriptionWhenUpdatePartitions()
.messageRouter(new MessageRouter() {
@Override
public int choosePartition(Message<?> msg, TopicMetadata metadata) {
return Integer.parseInt(msg.getKey()) % metadata.numPartitions();
return msg.hasKey() ? Integer.parseInt(msg.getKey()) % metadata.numPartitions() : 0;
Copy link
Contributor

Choose a reason for hiding this comment

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

this change looks unrelated to this patch, please revert

Copy link
Contributor Author

@equanz equanz Sep 2, 2021

Choose a reason for hiding this comment

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

I'll add this behavior for authn/authz (please see also #11570 (comment) ). Elect first partition index by MessageRouter with blank message.
https://github.com/equanz/pulsar/blob/844cca7cf84b76b7d27f12de4dbf36e23cbf29e9/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PartitionedProducerImpl.java#L87-L88

If we shouldn't change this test and behavior,

  1. add pseudo message key like xxx to blank message
  2. handle exceptions in PartitionedProducerImpl
  3. select first partition index without MessageRouter (e.g. randomly select)
    • if message router elects partition from part of partitions like singlepartition routing mode, create redundant producer for this change
  4. etc.

@@ -182,7 +182,7 @@ public void testPublishCompactAndConsumePartitionedTopics(Supplier<String> servi
.messageRouter(new MessageRouter() {
@Override
public int choosePartition(Message<?> msg, TopicMetadata metadata) {
return Integer.parseInt(msg.getKey()) % metadata.numPartitions();
return msg.hasKey() ? Integer.parseInt(msg.getKey()) % metadata.numPartitions() : 0;
Copy link
Contributor

Choose a reason for hiding this comment

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

why do you need to change this test ?

Comment on lines -116 to +134
return producers.get(0).getProducerName();
return producers.get(firstPartitionIndex).getProducerName();
Copy link
Contributor

Choose a reason for hiding this comment

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

shouldn't be the same producerName for all the producers ?
why not computing the name in the constructor and cache it ?

log.error("[{}] Could not create internal producer. partitionIndex: {}", topic, partition,
createException);
try {
producers.remove(partition).close();
Copy link
Contributor

Choose a reason for hiding this comment

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

I mean that we should test the result for "producers.remove(partition)" for equality ('==') with prod

@equanz
Copy link
Contributor Author

equanz commented Sep 2, 2021

@eolivelli
Thank you for your comments! I'll address your comments.
And could you please check my replies?

@equanz
Copy link
Contributor Author

equanz commented Sep 3, 2021

/pulsarbot run-failure-checks

Copy link
Contributor

@eolivelli eolivelli left a comment

Choose a reason for hiding this comment

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

I believe that the changes in the test cases are showing too many behaviour changes.

We should add a configuration option to enable this behaviour explicitly

@joefk @merlimat @rdhabalia @codelipenghui @sijie WDYT ?

Copy link
Contributor

@eolivelli eolivelli left a comment

Choose a reason for hiding this comment

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

Lgtm

@equanz
Copy link
Contributor Author

equanz commented Oct 12, 2021

/pulsarbot run-failure-checks

2 similar comments
@equanz
Copy link
Contributor Author

equanz commented Oct 12, 2021

/pulsarbot run-failure-checks

@equanz
Copy link
Contributor Author

equanz commented Oct 13, 2021

/pulsarbot run-failure-checks

@equanz
Copy link
Contributor Author

equanz commented Oct 13, 2021

@eolivelli Thank you for your review!

@massakam @k2la Rebased to resolve conflicts. Also, add fc83a5d commit to follow #12287 feature even if lazy-loading feature is enabled. PTAL.

@equanz
Copy link
Contributor Author

equanz commented Oct 15, 2021

/pulsarbot run-failure-checks

1 similar comment
@equanz
Copy link
Contributor Author

equanz commented Oct 15, 2021

/pulsarbot run-failure-checks

@equanz
Copy link
Contributor Author

equanz commented Oct 15, 2021

@massakam PTAL

@massakam massakam merged commit 1f8945a into apache:master Oct 15, 2021
@equanz equanz deleted the pip_79_lazy_loading branch October 15, 2021 06:38
bharanic-dev pushed a commit to bharanic-dev/pulsar that referenced this pull request Mar 18, 2022
…e#11570)

Fixes apache#11496 also matches part of PIP 79.

C++ implementation that closely matches the proposed Java client changes from reducing partitioned producer connections and lookups: PR 10279

### Motivation

Producers that send messages to partitioned topics start a producer per partition, even when using single partition routing. For topics that have the combination of a large number of producers and a large number of partitions, this can put strain on the brokers. With say 1000 partitions and single partition routing with non-keyed messages, 999 topic owner lookups and producer registrations are performed that could be avoided.

PIP 79 also describes this. I wrote this before realising that PIP 79 also covers this. This implementation can be reviewed and contrasted to the Java client implementation in apache#10279.

### Modifications

Allows partitioned producers to start producers for individual partitions lazily. Starting a producer involves a topic owner
lookup to find out which broker is the owner of the partition, then registering the producer for that partition with the owner
broker. For topics with many partitions and when using SinglePartition routing without keyed messages, all of these
lookups and producer registrations are a waste except for the single chosen partition.

This change allows the user to control whether a producer on a partitioned topic uses this lazy start or not, via a new config
in ProducerConfiguration. When ProducerConfiguration.setLazyStartPartitionedProducers(true) is set, the PartitionedProducerImpl.start() becomes a synchronous operation that only does housekeeping (no network operations).
The producer of any given partition is started (which includes a topic owner lookup and registration) upon sending the first message to that partition. While the producer starts, messages are buffered.

The sendTimeout timer is only activated once a producer has been fully started, which should give enough time for any buffered messages to be sent. For very short send timeouts, this setting could cause send timeouts during the start phase. The default of 30s should however not cause this issue.
bharanic-dev pushed a commit to bharanic-dev/pulsar that referenced this pull request Mar 18, 2022
…che#10279)

* feat: add lazy loading feature to PartitionedProducerImpl

* feat: add partial round robin routing mode

* test: add tests for lazy-loading

* fix: fix producer closing code at lazy-loading

* test: remove unnecessary handling, fail from test codes

* feat: add enableLazyStartPartitionedProducers config

* test: fix test for lazy-loading config

* fix: address comments

* fix: add partition-change interceptor

* fix: address comments
merlimat pushed a commit to apache/pulsar-client-python that referenced this pull request Sep 29, 2022
Fixes #11496 also matches part of PIP 79.

C++ implementation that closely matches the proposed Java client changes from reducing partitioned producer connections and lookups: PR 10279

### Motivation

Producers that send messages to partitioned topics start a producer per partition, even when using single partition routing. For topics that have the combination of a large number of producers and a large number of partitions, this can put strain on the brokers. With say 1000 partitions and single partition routing with non-keyed messages, 999 topic owner lookups and producer registrations are performed that could be avoided.

PIP 79 also describes this. I wrote this before realising that PIP 79 also covers this. This implementation can be reviewed and contrasted to the Java client implementation in apache/pulsar#10279.

### Modifications

Allows partitioned producers to start producers for individual partitions lazily. Starting a producer involves a topic owner
lookup to find out which broker is the owner of the partition, then registering the producer for that partition with the owner
broker. For topics with many partitions and when using SinglePartition routing without keyed messages, all of these
lookups and producer registrations are a waste except for the single chosen partition.

This change allows the user to control whether a producer on a partitioned topic uses this lazy start or not, via a new config
in ProducerConfiguration. When ProducerConfiguration.setLazyStartPartitionedProducers(true) is set, the PartitionedProducerImpl.start() becomes a synchronous operation that only does housekeeping (no network operations).
The producer of any given partition is started (which includes a topic owner lookup and registration) upon sending the first message to that partition. While the producer starts, messages are buffered.

The sendTimeout timer is only activated once a producer has been fully started, which should give enough time for any buffered messages to be sent. For very short send timeouts, this setting could cause send timeouts during the start phase. The default of 30s should however not cause this issue.
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
area/client type/enhancement The enhancements for the existing features or docs. e.g. reduce memory usage of the delayed messages type/feature The PR added a new feature or issue requested a new feature type/PIP
Projects
None yet
Development

Successfully merging this pull request may close these issues.

8 participants