Thanks for the suggestions Matteo and Rajan. I've created a bundled
namespace (16 bundles) and a partitioned topic (8 partitions). However, I'm
stilling running into issues running perf tests. Client-side, I'm
continuously seeing the following exception:

Caused by: java.util.concurrent.CompletionException:
org.apache.pulsar.client.api.PulsarClientException$LookupException:
java.lang.IllegalStateException: Namespace bundle
test/us-west/ns-bundle/0x50000000_0x60000000 is being unloaded

Server-side, I see the following error:

2017-10-11 18:19:07,978 - INFO  [pulsar-web-56-3:Slf4jRequestLog@60] -
172.31.10.179 - - [11/Oct/2017:18:19:07 +0000] "PUT
//ip-172-31-10-179.us-west-2.compute.internal:8080/admin/namespaces/test/us-west/ns-bundle/0x50000000_0x58000000/split
HTTP/1.1" 500 5278 "-" "Jersey/2.23.2 (HttpUrlConnection 1.8.0_141)" 4
2017-10-11 18:19:07,979 - ERROR
[pulsar-load-manager-11-1:SimpleLoadManagerImpl@1455] - *Failed to split
namespace bundle test/us-west/ns-bundle/0x50000000_0x58000000*
org.apache.pulsar.client.admin.PulsarAdminException$ServerSideErrorException:
Some error occourred on the server
[trace redacted]
Caused by: *javax.ws.rs.InternalServerErrorException: HTTP 500 Internal
Server Error*
[trace redacted]


I can provide the stack traces if needed. I'm not seeing any WARN logs in
the bookies.

On Tue, Oct 10, 2017 at 5:45 PM, Rajan Dhabalia <[email protected]>
wrote:

> >> I have no idea what "0x00000000_0xffffffff" is or why it's being used
> in place of the topic name I've given.
>
> 0x00000000_0xffffffff defines the bundle-range.
> Namespace can be divided into multiple logical parts by defining bundle
> range. Initially, by default every namespace has 1 bundle with range:
> "0x00000000_0xffffffff".
> If you split it into 2 bundles then this bundle-range will be :
> "0x00000000_0x7FFFFFFF" and "0x7FFFFFFF_0xFFFFFFFF". and based on
> topic-name's hash, that topic will fall under  appropriate bundle. Broker
> which owns that bundle, will own all topics that fall under that
> namespace-bundle.
>
> To split bundle, you have to first create a namespace which creates a
> namespace-metadata place-holder in zookeeper. So, we can't split namespace
> bundle if namespace is not created.
>
> >> I'll try out the ModularLoadManager.
> Sure, ModularLoadManager has visibility of larger metrics of broker's load
> and it distributes load efficiently. However, ModularLoadManager doesn't
> support auto-split functionality right now and PR
> <https://github.com/apache/incubator-pulsar/pull/385> is open. Probably
> ModularLoadManager's auto-split functionality will be available by next
> release.
>
> Thanks,
> Rajan
>
>
>
> On Tue, Oct 10, 2017 at 5:13 PM, Ryan Stout <[email protected]> wrote:
>
>> I should've looked before, as I do see exceptions in the logs due to
>> bundle splits. It's complaining about a missing namespace, however I'm able
>> to successfully publish to the topic 
>> "persistent://test/us-west/ns1/p4-topic".
>> I have no idea what "0x00000000_0xffffffff" is or why it's being used in
>> place of the topic name I've given.
>>
>> I'll try out the ModularLoadManager.
>>
>> Logs:
>>
>> 2017-10-11 00:06:04,412 - INFO  [pulsar-load-manager-11-1:Simp
>> leLoadManagerImpl@1398] - Running namespace bundle split with
>> thresholds: topics 1000, sessions 1000, msgRate 1000, bandwidth 104857600,
>> maxBundles 128
>> 2017-10-11 00:06:04,413 - INFO  [pulsar-load-manager-11-1:Simp
>> leLoadManagerImpl@1435] - Will split hot namespace bundle
>> test/us-west/ns1/0x00000000_0xffffffff, topics 4, producers+consumers 8,
>> msgRate in+out 1999.1277760920889, bandwidth in+out 2121007.929782623
>> 2017-10-11 00:06:04,414 - INFO  [pulsar-simple-load-manager-55
>> -1:SimpleLoadManagerImpl@698] - doLoadRanking - load balancing strategy:
>> weightedRandomSelection
>> 2017-10-11 00:06:04,416 - INFO  [pulsar-web-56-14:Namespaces@789] -
>> [null] Split namespace bundle test/us-west/ns1/0x00000000_0xffffffff
>> 2017-10-11 00:06:04,418 - INFO  [pulsar-web-56-14:Slf4jRequestLog@60] -
>> 172.31.10.179 - - [11/Oct/2017:00:06:04 +0000] "PUT
>> //ip-172-31-10-179.us-west-2.compute.internal:8080/admin/nam
>> espaces/test/us-west/ns1/0x00000000_0xffffffff/split HTTP/1.1" 404 37
>> "-" "Jersey/2.23.2 (HttpUrlConnection 1.8.0_141)" 3
>> 2017-10-11 00:06:04,419 - *ERROR* [pulsar-load-manager-11-1:Simp
>> leLoadManagerImpl@1455] - *Failed to split namespace bundle
>> test/us-west/ns1/0x00000000_0xffffffff*
>> org.apache.pulsar.client.admin.*PulsarAdminException$NotFoundException:
>> Namespace does not exist*
>> at org.apache.pulsar.client.admin.internal.BaseResource.getApiE
>> xception(BaseResource.java:173)
>> at org.apache.pulsar.client.admin.internal.NamespacesImpl.split
>> NamespaceBundle(NamespacesImpl.java:352)
>> at org.apache.pulsar.broker.loadbalance.impl.SimpleLoadManagerI
>> mpl.doNamespaceBundleSplit(SimpleLoadManagerImpl.java:1450)
>> at org.apache.pulsar.broker.loadbalance.impl.SimpleLoadManagerI
>> mpl.writeLoadReportOnZookeeper(SimpleLoadManagerImpl.java:1271)
>> at org.apache.pulsar.broker.loadbalance.LoadReportUpdaterTask.
>> run(LoadReportUpdaterTask.java:41)
>> at java.util.concurrent.Executors$RunnableAdapter.call(
>> Executors.java:511)
>> at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308)
>> at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFu
>> tureTask.access$301(ScheduledThreadPoolExecutor.java:180)
>> at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFu
>> tureTask.run(ScheduledThreadPoolExecutor.java:294)
>> at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPool
>> Executor.java:1149)
>> at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoo
>> lExecutor.java:624)
>> at io.netty.util.concurrent.DefaultThreadFactory$DefaultRunnabl
>> eDecorator.run(DefaultThreadFactory.java:144)
>> at java.lang.Thread.run(Thread.java:748)
>> Caused by: javax.ws.rs.NotFoundException: HTTP 404 Not Found
>> at org.glassfish.jersey.client.JerseyInvocation.convertToExcept
>> ion(JerseyInvocation.java:1020)
>> at org.glassfish.jersey.client.JerseyInvocation.translate(Jerse
>> yInvocation.java:819)
>> at org.glassfish.jersey.client.JerseyInvocation.access$700(Jers
>> eyInvocation.java:92)
>> at org.glassfish.jersey.client.JerseyInvocation$2.call(JerseyIn
>> vocation.java:701)
>> at org.glassfish.jersey.internal.Errors.process(Errors.java:315)
>> at org.glassfish.jersey.internal.Errors.process(Errors.java:297)
>> at org.glassfish.jersey.internal.Errors.process(Errors.java:228)
>> at org.glassfish.jersey.process.internal.RequestScope.runInScop
>> e(RequestScope.java:444)
>> at org.glassfish.jersey.client.JerseyInvocation.invoke(JerseyIn
>> vocation.java:697)
>> at org.glassfish.jersey.client.JerseyInvocation$Builder.method(
>> JerseyInvocation.java:448)
>> at org.glassfish.jersey.client.JerseyInvocation$Builder.put(Jer
>> seyInvocation.java:332)
>> at org.apache.pulsar.client.admin.internal.NamespacesImpl.split
>> NamespaceBundle(NamespacesImpl.java:350)
>> ... 11 more
>>
>>
>> On Tue, Oct 10, 2017 at 4:44 PM, Rajan Dhabalia <[email protected]>
>> wrote:
>>
>>> COUNT          |TOPIC          |BUNDLE         |PRODUCER
>>>  |CONSUMER       |BUNDLE +       |BUNDLE -
>>>                              4              |1                     |8
>>>                       |0                          |0
>>> |0              ||
>>>
>>> ip-[redacted].us-west-2.compute.internal:8080             |1
>>> |1500.41      |639.99       |3414.49      |15.97        ||
>>>
>>>
>>> Based on stats, it seems : a broker is serving 4 topics under the same
>>> bundle. So, yes, we need to split the bundle so, topics can be distributed
>>> evenly into multiple bundles and those bundles can be owned by different
>>> brokers. There are few pointers to troubleshoot bundle-splitting:
>>>
>>> *1. Is there any way to verify if bundle is split automatically by
>>> loadbalancer in the log?*
>>> In the broker log under class: *SimpleLoadManagerImpl* do you seen any
>>> log with text
>>>
>>> *"split hot namespace bundle"?*
>>> *2. Is there any way to split the bundle manually and unload namespace
>>> bundles?*
>>>   A. we can split bundle manually using pulsar-admin tool
>>> <https://pulsar.incubator.apache.org/docs/latest/admin-api/namespaces/#splitbundle>
>>>
>>> pulsar-admin namespaces split-bundle --bundle 0x00000000_0xffffffff 
>>> test-property/cl1/ns1
>>>
>>>  B. Unload namespace bundle
>>>
>>> pulsar-admin namespaces unload --bundle 0x00000000_0xffffffff 
>>> test-property/pstg-gq1/ns1
>>>
>>>
>>> *3. How to get list of bundles which my broker is serving?*
>>>
>>> pulsar-admin namespaces broker-stats destinations -i
>>> {
>>>     "sample/standalone/ns1": {
>>>         "0x00000000_0xffffffff": {
>>>             "persistent": {
>>>                 "persistent://sample/standalone/ns1/t1": {
>>>                     "publishers": [],
>>>                     "replication": {},
>>>                     "subscriptions": {},
>>>                     "producerCount": 0,
>>>                     "averageMsgSize": 0.0,
>>>                     "msgRateIn": 0.0,
>>>                     "msgRateOut": 0.0,
>>>                     "msgThroughputIn": 0.0,
>>>                     "msgThroughputOut": 0.0,
>>>                     "storageSize": 0,
>>>                     "pendingAddEntriesCount": 0
>>>                 }
>>>             }
>>>         }
>>>     }
>>>
>>>
>>> *this commands gives list of namespace-bundles, topics and its output.*
>>>
>>>
>>> *4. Few release back, there is an advanced load-balancer is introduced
>>> in pulsar which does better job in terms of distributing load. How can we
>>> enable new advanced load-balancer?*
>>> Modular-load-manager
>>> <https://pulsar.incubator.apache.org/docs/latest/admin/ModularLoadManager/>
>>>
>>> Thanks,
>>> Rajan
>>>
>>> On Tue, Oct 10, 2017 at 4:04 PM, Ryan Stout <[email protected]> wrote:
>>>
>>>> I've created a topic with 4 partitions, and monitor-brokers reports 4
>>>> topics:
>>>>
>>>> ============================================================
>>>> =======================================================
>>>> ||COUNT          |TOPIC          |BUNDLE         |PRODUCER
>>>>  |CONSUMER       |BUNDLE +       |BUNDLE -       ||
>>>> ||               |4              |1              |8              |0
>>>>           |0              |0              ||
>>>> ||RAW SYSTEM     |CPU %          |MEMORY %       |DIRECT %       |BW IN
>>>> %        |BW OUT %       |MAX %          ||
>>>> ||               |2.95           |18.36          |1.56           |0.16
>>>>          |0.29           |18.36          ||
>>>> ||ALLOC SYSTEM   |CPU %          |MEMORY %       |DIRECT %       |BW IN
>>>> %        |BW OUT %       |MAX %          ||
>>>> ||               |42.68          |3.88           |               |3.57
>>>>          |2.90           |42.68          ||
>>>> ||RAW MSG        |MSG/S IN       |MSG/S OUT      |TOTAL          |KB/S
>>>> IN        |KB/S OUT       |TOTAL          ||
>>>> ||               |1500.41        |0.00           |1500.41
>>>> |16.14          |29.18          |45.32          ||
>>>> ||ALLOC MSG      |MSG/S IN       |MSG/S OUT      |TOTAL          |KB/S
>>>> IN        |KB/S OUT       |TOTAL          ||
>>>> ||               |3295.35        |118.70         |3414.05
>>>> |357.11         |289.76         |646.86         ||
>>>> ============================================================
>>>> =======================================================
>>>>
>>>> I also see a throughput of over 1k on one of the brokers:
>>>>
>>>> 2017-10-10 21:16:25,548 - INFO  - [main:BrokerMonitor@203] - Overall
>>>> Broker Data:
>>>> ************************************************************
>>>> ************************************************************
>>>> ***************
>>>> ||BROKER                                                       |BUNDLE
>>>>      |MSG/S        |LONG/S       |KB/S         |MAX %        ||
>>>> ||ip-[redacted].us-west-2.compute.internal:8080              |0
>>>>     |0.00         |0.00         |0.00         |5.81         ||
>>>> ||ip-[redacted].us-west-2.compute.internal:8080             |1
>>>>     |1500.41      |639.99       |3414.49      |15.97        ||
>>>> ||TOTAL                                                        |1
>>>>       |1500.41      |3414.49      |639.99       |15.97        ||
>>>> ************************************************************
>>>> ************************************************************
>>>> ***************
>>>>
>>>>
>>>> On Tue, Oct 10, 2017 at 3:48 PM, Rajan Dhabalia <[email protected]>
>>>> wrote:
>>>>
>>>>> Hi Ryan,
>>>>>
>>>>> >> I've set "loadBalancerAutoBundleSplitEnabled" to "true" and
>>>>> "loadBalancerNamespaceBundleMaxMsgRate" to 1000. I then ran 2
>>>>> producers at 1k msg/s for ~5mins, but I didn't see a bundle split
>>>>>
>>>>> LoadBalancer will split the bundle only if it has more than 1 topic in
>>>>> the bundle (because bundle is a logical part of namespace that contains
>>>>> topics. if namespace has only 1 topic then there is no need of split
>>>>> bundle).
>>>>> Load-balancer splits bundle if bundle reaches one of the threshold
>>>>> configured at broker-config
>>>>> <https://git.corp.yahoo.com/cloud-messaging/pulsar/blob/yahoo/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java#L260-L266>:
>>>>>
>>>>> 1. *loadBalancerNamespaceBundleMaxTopics*:
>>>>> maximum topics in a bundle
>>>>> 2. *loadBalancerNamespaceBundleMaxSessions*:                 maximum
>>>>> sessions (producers + consumers) in a bundle
>>>>> 3. *loadBalancerNamespaceBundleMaxMsgRate*:                  maximum
>>>>> msgRate (in + out) in a bundle
>>>>> 4. *loadBalancerNamespaceBundleMaxBandwidthMbytes*:   maximum
>>>>> bandwidth (in + out) in a bundle
>>>>>
>>>>> >> I found "bin/pulsar-perf monitor-brokers"
>>>>> Using this utility can you confirm bundle usage and can you confirm if
>>>>> it meets that threshold to split the bundle?
>>>>>
>>>>> Thanks,
>>>>> Rajan
>>>>>
>>>>>
>>>>>
>>>>> On Tue, Oct 10, 2017 at 3:33 PM, Ryan Stout <[email protected]> wrote:
>>>>>
>>>>>> Hey Pulsar devs,
>>>>>>
>>>>>> I've deployed a small Pulsar cluster (in AWS) with 2 brokers and 3
>>>>>> bookies. I've started doing perf testing using bin/pulsar-perf to 
>>>>>> determine
>>>>>> the limitations of Pulsar. I'm at the point where I can't produce more 
>>>>>> than
>>>>>> ~25k msg/s on a topic (regardless of number of partitions, clients, or
>>>>>> bookies). Upon trying to understand the bottleneck, I found
>>>>>> "bin/pulsar-perf monitor-brokers" and it showed that only one of the two
>>>>>> brokers is receiving traffic. I've set-up the service-discovery service
>>>>>> that came with Pulsar, which my producers are hitting, so I expected the
>>>>>> requests to be distributed fairly across the brokers, but this is not the
>>>>>> case.
>>>>>>
>>>>>> In conf/broker.conf, there's a load balancing section that seems to
>>>>>> hint at the ability for brokers to shed traffic to other brokers. I've
>>>>>> tried tuning the values in this section, but haven't been able to get the
>>>>>> brokers to share the load. For example, I've set
>>>>>> "loadBalancerAutoBundleSplitEnabled" to "true" and
>>>>>> "loadBalancerNamespaceBundleMaxMsgRate" to 1000. I then ran 2
>>>>>> producers at 1k msg/s for ~5mins, but I didn't see a bundle split (I also
>>>>>> reduced some of the intervals e.g. "loadBalancerSheddingIntervalMinutes"
>>>>>> to 1 minute).
>>>>>>
>>>>>> Is there a way to configure my Pulsar cluster to balance between my 2
>>>>>> brokers? Is there perhaps another, better way I might increase 
>>>>>> throughput?
>>>>>>
>>>>>
>>>>>
>>>>
>>>
>>
>

Reply via email to