-
Notifications
You must be signed in to change notification settings - Fork 3.6k
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
[improve][broker] Register the broker to metadata store without version id compare #23298
[improve][broker] Register the broker to metadata store without version id compare #23298
Conversation
pulsar-metadata/src/main/java/org/apache/pulsar/metadata/cache/impl/MetadataCacheImpl.java
Outdated
Show resolved
Hide resolved
@BewareMyPower But it could happen from user's incorrect configuration that more than 1 brokers use the same broker id especially for a on-prem environment. And the comment #23297 (comment) is also the same concern about the lock issue. It looks like something cause the lock didn't get released even if the broker unregister itself from the metadata server. |
If multiple brokers have the same broker id, the configuration would have multiple other problems. Wouldn't it be a valid requirement that the admin of the Pulsar system ensures that broker ids are unique?
Yes, that seems to be a real issue that would need to be addressed in any case. |
I agree with Lari that we should not guarantee a certain behavior with wrong configuration. If you assumed invalid configuration could happen, what about two brokers configured different metadata stores? How can you prevent the broker with wrong metadata URL from starting? |
From another perspective, this PR resolves the case the broker exited unexpectedly (e.g. by a crash). Just like the incident we met online:
The JVM process exited by a SIGSEGV signal and no cleanup was done, even the metadata store client was not closed. Then when the broker restarted, it would fail with When programming, we should handle exceptions rather than errors.
There is another solution that we can add retry logic to final var deadline = System.currentTimeMillis() + pulsar.getConfig().getMetadataStoreSessionTimeoutMillis();
Throwable throwable = null;
while (true) {
if (System.currentTimeMillis() > deadline) {
throw MetadataStoreException.unwrap(throwable == null ? new TimeoutException("Failed to register")
: throwable);
}
try {
this.brokerLookupDataLock = brokerLookupDataLockManager.acquireLock(keyPath(brokerId), brokerLookupData)
.get(conf.getMetadataStoreOperationTimeoutSeconds(), TimeUnit.SECONDS);
this.state = State.Registered;
break;
} catch (InterruptedException | ExecutionException | TimeoutException e) {
throwable = e;
try {
Thread.sleep(100);
} catch (InterruptedException ex) {
throw MetadataStoreException.unwrap(ex);
}
}
} |
I just thought it again. Retry is not a good solution. If the broker crashed, the registered data could still exist in the metadata store so that this "dead" broker could be selected as the owner. However, even after it restarted, it could not serve any request until the metadata store session timeout. So many topics could be unavailable for a long time. Therefore, it does not make sense to keep the current flaky logic just to fail fast for a misconfiguration case. @codelipenghui |
...broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/BrokerRegistryImpl.java
Outdated
Show resolved
Hide resolved
I'm not going to say we should handle the incorrect configuration from the user side, but what is the expected behavior if user made an incorrect broker ID (same broker ID)? As I understand, only the first broker should register successfully, all the other brokers with the same broker ID should get failed? |
pulsar-metadata/src/main/java/org/apache/pulsar/metadata/api/MetadataCache.java
Show resolved
Hide resolved
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
LGTM
In this case, one of the benefits of this change is that in the case of broker crash, that the broker can recover immediately. The session of the crashed instance will block starting of a new instance if we don't make this change. I think that it's a different matter to detect if someone has duplicate broker ids in their Pulsar cluster configuration. @BewareMyPower Perhaps the description could be updated to include the actual reason why we want to do this change, so that a crashed instance can recover immediately, without needing to wait until the session of the crashed instance expires? |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Do we need to apply the same change for ModularLoadManager
?
@lhotari Updated.
@codelipenghui I'd like to say, undefined behavior. Kafka's behavior is also that only one broker will succeed if multiple brokers configure the same broker.id. However, this behavior is never documented. It only suggests generating broker id automatically by ZooKeeper:
Actually, in real world, only if two brokers were configured with the same It would be good if users can quickly know they've configured the same broker id for different brokers. However, IMO, this issue caused by misconfiguration can hardly exist in practical rather than in theory. |
@Demogorgon314 +1 to me |
I swallowed the exception in BTW, I think all synchronous close operations for components should not throw any exception. Otherwise, the following close operations will be skipped. Just add error logs for unexpected exceptions and warning logs for expected exceptions. |
Codecov ReportAttention: Patch coverage is
Additional details and impacted files@@ Coverage Diff @@
## master #23298 +/- ##
============================================
+ Coverage 73.57% 74.56% +0.99%
- Complexity 32624 34308 +1684
============================================
Files 1877 1927 +50
Lines 139502 145040 +5538
Branches 15299 15860 +561
============================================
+ Hits 102638 108150 +5512
+ Misses 28908 28617 -291
- Partials 7956 8273 +317
Flags with carried forward coverage won't be shown. Click here to find out more.
|
Motivation
It's observed in our test environment many times after restarting the only broker with the extensible load manager's broker registry service.
The "Resource xxx is already locked" is usually caused by a
BadVersionException
. This issue can happen when:kill -9
or the container crashed) so it did not delete the node from the metadata store.Actually, it's not a good use case of
LockManager
here. The broker registry only:/loadbalance/brokers/<broker-id>
when starting (in the main thread)/loadbalance/brokers/<broker-id>
/loadbalance/brokers
.Currently, step 1 and 2 are performed by the distribution lock implemented by
LockManager
. However, there should be no race condition because the broker id is the unique identifier of a broker. There is no case that a broker starts or closes concurrently. In addition, it's meaningless to pass a version explicitly to the metadata store.Regarding step 3 and 4, they are simply wrapping the
MetadataCache
's methods. They are actually never related to something like "lock" though they're namedlistLocks
andreadLock
.Modifications
put
method toMetadataCache
, which just callsMetadataStore#put
withOptional.empty()
underlying.LockManager
withMetadataCache
inBrokerRegistryImpl
.Documentation
doc
doc-required
doc-not-needed
doc-complete
Matching PR in forked repository
PR in forked repository: