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

[fix][broker] Avoid being stuck in 30+ seconds when closing the BrokerService #31

Open
wants to merge 2 commits into
base: master
Choose a base branch
from

Conversation

BewareMyPower
Copy link
Owner

Fixes apache#22569

Motivation

BrokerService#closeAsync calls unloadNamespaceBundlesGracefully to unload namespaces gracefully. With extensible load manager, it eventually calls TableViewLoadDataStoreImpl#validateProducer:

BrokerService#unloadNamespaceBundlesGracefully
  ExtensibleLoadManagerWrapper#disableBroker
    ExtensibleLoadManagerImpl#disableBroker
      ServiceUnitStateChannelImpl#cleanOwnerships
        ServiceUnitStateChannelImpl#doCleanup
          TableViewLoadDataStoreImpl#removeAsync
            TableViewLoadDataStoreImpl#validateProducer

In validateProducer, if the producer is not connected, it will recreate the producer synchronously. However, since the state of PulsarService has already been changed to Closing, all connect or lookup requests will fail with ServiceNotReady. Then the client will retry until timeout.

Besides, the unload operation could also trigger the reconnection because the extensible load manager sends the unload event to the loadbalancer-service-unit-state topic.

Modifications

The major fix:
Before changing PulsarService's state to Closing, call BrokerService#unloadNamespaceBundlesGracefully first to make the load manager complete the unload operations first.

Minor fixes:

  • Record the time when LoadManager#disableBroker is done.
  • Don't check if producer is disconnected because the producer could retry if it's disconnected.

Verifications

Add ExtensibleLoadManagerCloseTest to verify closing PulsarService won't take too much time. Here are some test results locally:

2024-04-24T19:43:38,851 - INFO  - [main:ExtensibleLoadManagerCloseTest] - Brokers close time: [3342, 3276, 3310]
2024-04-24T19:44:26,711 - INFO  - [main:ExtensibleLoadManagerCloseTest] - Brokers close time: [3357, 3258, 3298]
2024-04-24T19:46:16,791 - INFO  - [main:ExtensibleLoadManagerCloseTest] - Brokers close time: [3313, 3257, 3263]
2024-04-24T20:13:05,763 - INFO  - [main:ExtensibleLoadManagerCloseTest] - Brokers close time: [3304, 3279, 3299]
2024-04-24T20:13:43,979 - INFO  - [main:ExtensibleLoadManagerCloseTest] - Brokers close time: [3343, 3308, 3310]

As you can see, each broker takes only about 3 seconds to close due to OWNERSHIP_CLEAN_UP_CONVERGENCE_DELAY_IN_MILLIS value added in apache#20315

…rService

Fixes apache#22569

### Motivation

`BrokerService#closeAsync` calls `unloadNamespaceBundlesGracefully` to
unload namespaces gracefully. With extensible load manager, it
eventually calls `TableViewLoadDataStoreImpl#validateProducer`:

```
BrokerService#unloadNamespaceBundlesGracefully
  ExtensibleLoadManagerWrapper#disableBroker
    ExtensibleLoadManagerImpl#disableBroker
      ServiceUnitStateChannelImpl#cleanOwnerships
        ServiceUnitStateChannelImpl#doCleanup
          TableViewLoadDataStoreImpl#removeAsync
            TableViewLoadDataStoreImpl#validateProducer
```

In `validateProducer`, if the producer is not connected, it will
recreate the producer synchronously. However, since the state of
`PulsarService` has already been changed to `Closing`, all connect or
lookup requests will fail with `ServiceNotReady`. Then the client will
retry until timeout.

Besides, the unload operation could also trigger the reconnection
because the extensible load manager sends the unload event to the
`loadbalancer-service-unit-state` topic.

### Modifications

The major fix:
Before changing PulsarService's state to `Closing`, call
`BrokerService#unloadNamespaceBundlesGracefully` first to make the load
manager complete the unload operations first.

Minor fixes:
- Record the time when `LoadManager#disableBroker` is done.
- Don't check if producer is disconnected because the producer could
  retry if it's disconnected.

### Verifications

Add `ExtensibleLoadManagerCloseTest` to verify closing `PulsarService`
won't take too much time. Here are some test results locally:

```
2024-04-24T19:43:38,851 - INFO  - [main:ExtensibleLoadManagerCloseTest] - Brokers close time: [3342, 3276, 3310]
2024-04-24T19:44:26,711 - INFO  - [main:ExtensibleLoadManagerCloseTest] - Brokers close time: [3357, 3258, 3298]
2024-04-24T19:46:16,791 - INFO  - [main:ExtensibleLoadManagerCloseTest] - Brokers close time: [3313, 3257, 3263]
2024-04-24T20:13:05,763 - INFO  - [main:ExtensibleLoadManagerCloseTest] - Brokers close time: [3304, 3279, 3299]
2024-04-24T20:13:43,979 - INFO  - [main:ExtensibleLoadManagerCloseTest] - Brokers close time: [3343, 3308, 3310]
```

As you can see, each broker takes only about 3 seconds to close due to
`OWNERSHIP_CLEAN_UP_CONVERGENCE_DELAY_IN_MILLIS` value added in
apache#20315
@BewareMyPower
Copy link
Owner Author

Failed tests:

ModularLoadManagerImplTest.shutdown:240 » PulsarServer java.lang.NullPointerException: Cannot invoke "org.apache.pulsar.broker.service.BrokerService.unloadNamespaceBundlesGracefully()" because "<local2>.brokerService" is null
  Error:  org.apache.pulsar.broker.PulsarServiceTest.testBacklogAndRetentionCheck  Time elapsed: 16.407 s  <<< FAILURE!
  org.apache.pulsar.broker.PulsarServerException: java.lang.NullPointerException: Cannot invoke "org.apache.pulsar.broker.service.BrokerService.unloadNamespaceBundlesGracefully()" because "<local2>.brokerService" is null

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

[Bug] Broker could take 30+ seconds to close with extensible load manager
1 participant