2018-03-30 16:27:53 UTC - Joe Francis: I am for a topic level throttle.
Connection level throttling doesn't seem to have any real world semantics -
what does it mean to a user ?
----
2018-03-30 16:50:17 UTC - Matteo Merli: > Once the rateLimit is hit the
Broker sends this command to all producers connected to the topic and expects
clients to reject all newly published messages for stopPublishingInMs
..so that will break ordering
> required uint64 stopPublishingInMs = 3;
What the broker could give is an “hint” on what the rate can be for this
particular producer. Eg: if 10 producers are connected and limit is 1K, broker
could advertise 100/s each.
> I am for a topic level throttle. Connection level throttling doesn’t
seem to have any real world semantics - what does it mean to a user ?
One thing is the goal to achieve and how to configure it. Another is how to
reach that goal. Slowing down a connection (when the limit per-topic is
reached) is probably the most effective way to deal with the problem
----
2018-03-30 19:23:21 UTC - Jai Asher: > ..so that will break ordering
No cause we will also fail all pending acks when we fail a message - fail
entire pending message queue, just like a reconnect scenario
So from what I understand we are in general consensus about using the "Broker
side connection level throttling - server side limit", which is similar to the
currnet implementation.
As for "Client Side backoff - soft throttling" building on what Rajan and
Matteo suggested
- We have parameter `publishThrottlingRatePerTopicInMsg` like discussed
before.
- The Broker decides what rates a producer can produce at and sends the
new rates every X seconds (Configurable).
- Create a new command
```
CommandNewRates {
required uint64 producer_id = 1;
required string producer_name = 2;
// time when next permit will be available + a small random
offset.
required double publishRate = 3;
}
```
- Enhance `CommandProducerSuccess` to add
`optional double publishRate = 3;`
- The Client ratelimits the Publishes based on those rate.
For Example:-
Let publishThrottlingRatePerTopicInMsg=1000 MPS
Let publishRateRefreshTime=5 seconds - broker will send CommandNewRates every
five seconds
Time - 0 seconds
When producer-1 connects we send the rate as 1000 MPS in CommandProducerSuccess
When producer-2 connects we send the rate as 1000 MPS in CommandProducerSuccess
When producer-3 connects we send the rate as 1000 MPS in CommandProducerSuccess
When producer-4 connects we send the rate as 1000 MPS in CommandProducerSuccess
Basically we trust all producers to behave just as we do currently.
Time - 5 seconds
producer-1 produced at 1000 MPS
producer-2 produced at 0 MPS
producer-3 producer at 5000 MPS
producer-4 producer at 10000 MPS
Broker calculates new rates -
Average rate = publishThrottlingRatePerTopicInMsg / total #producers = 1000/4
= 2500 MPS
Unused publish rate => Average rate - producer publish rate
From producer-1 => max(0, 2500 - 1000) => 1500
From producer-2 => max(0, 2500 - 0) => 2500
From producer-3 => max(0, 2500 - 5000) => 0
From producer-4 => max(0, 2500 - 10000) => 0
Total Unused publish rate => 4000 MPS
Over used publish rate => producer publish rate - Average rate
From producer-1 => max(0, 1000 - 2500) => 0
From producer-2 => max(0, 0 - 2500) => 0
From producer-3 => max(0, 5000 - 2500) => 2500
From producer-4 => max(0, 10000 - 2500) => 7500
Total Over used publish rate => 10000 MPS
Ratio of over used publish rate => Over used publish rate / Total Over used
publish rate
From producer-1 => 0 / 10000 = 0
From producer-2 => 0 / 10000 = 0
From producer-3 => 2500 / 10000 = 0.25
From producer-4 => 7500 / 10000 = 0.75
New Publish rate = Average rate + (Ratio of over used publish rate * Total
Unused publish rate)
For producer-1 => 2500 + 0 * 4000 => 2500 MPS
For producer-2 => 2500 + 0 * 4000 => 2500 MPS
For producer-3 => 2500 + 0.25 * 4000 => 3500 MPS
For producer-4 => 2500 + 0.75 * 4000 => 5500 MPS
Broker sends new publish rates to producers and expects the clients to adhere
to it
If the clients don't obey the broker throttles the connection `setAutoRead off`
till next permit is available, as discussed above in `Broker side connection
level throttling`
This makes sures that only the connections of misbehaving producers are
throttled.
==============
Basically if a producer behaves i.e is within average publish rate (2500) - we
trust it to continue behaving and distribute its unused rate to other producers.
==============
Do you think this approach is valid enough to start off with a PIP?
----
2018-03-31 00:04:48 UTC - Rajan Dhabalia: - Pulsar master is pointing to:
`<bookkeeper.version>4.7.0-SNAPSHOT</bookkeeper.version>`
is it using same Apache Bookkeeper version?
<https://github.com/apache/bookkeeper>
I am getting error:
```
java.lang.AbstractMethodError: Method
com/myauth/ClientAuthProviderFactory.newProvider(Lorg/apache/bookkeeper/proto/ClientConnectionPeer;Lorg/apache/bookkeeper/auth/AuthCallbacks$GenericCallback;)Lorg/apache/bookkeeper/auth/ClientAuthProvider;
is abstract
```
Also, Pulsar commits are full of `Pulsar-Function-PIP`.. it seems
Pulsar-Function-PIP had large number of commits which is making hard to find
previous history commit.. I think Pulsar is following: "Squash and merge" ?
----