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

[improve][broker] Register the broker to metadata store without version id compare #23298

Merged
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -39,11 +39,10 @@
import org.apache.pulsar.broker.ServiceConfiguration;
import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLookupData;
import org.apache.pulsar.common.util.FutureUtil;
import org.apache.pulsar.metadata.api.MetadataCache;
import org.apache.pulsar.metadata.api.MetadataStoreException;
import org.apache.pulsar.metadata.api.Notification;
import org.apache.pulsar.metadata.api.NotificationType;
import org.apache.pulsar.metadata.api.coordination.LockManager;
import org.apache.pulsar.metadata.api.coordination.ResourceLock;

/**
* The broker registry impl, base on the LockManager.
Expand All @@ -57,16 +56,14 @@ public class BrokerRegistryImpl implements BrokerRegistry {

private final BrokerLookupData brokerLookupData;

private final LockManager<BrokerLookupData> brokerLookupDataLockManager;
private final MetadataCache<BrokerLookupData> brokerLookupDataMetadataCache;

private final String brokerId;
private final String brokerIdKeyPath;

private final ScheduledExecutorService scheduler;

private final List<BiConsumer<String, NotificationType>> listeners;

private volatile ResourceLock<BrokerLookupData> brokerLookupDataLock;

protected enum State {
Init,
Started,
Expand All @@ -79,10 +76,10 @@ protected enum State {
public BrokerRegistryImpl(PulsarService pulsar) {
this.pulsar = pulsar;
this.conf = pulsar.getConfiguration();
this.brokerLookupDataLockManager = pulsar.getCoordinationService().getLockManager(BrokerLookupData.class);
this.brokerLookupDataMetadataCache = pulsar.getLocalMetadataStore().getMetadataCache(BrokerLookupData.class);
this.scheduler = pulsar.getLoadManagerExecutor();
this.listeners = new ArrayList<>();
this.brokerId = pulsar.getBrokerId();
this.brokerIdKeyPath = keyPath(pulsar.getBrokerId());
this.brokerLookupData = new BrokerLookupData(
pulsar.getWebServiceAddress(),
pulsar.getWebServiceAddressTls(),
Expand Down Expand Up @@ -122,7 +119,7 @@ public boolean isStarted() {
public synchronized void register() throws MetadataStoreException {
if (this.state == State.Started) {
try {
this.brokerLookupDataLock = brokerLookupDataLockManager.acquireLock(keyPath(brokerId), brokerLookupData)
brokerLookupDataMetadataCache.put(brokerIdKeyPath, brokerLookupData)
BewareMyPower marked this conversation as resolved.
Show resolved Hide resolved
.get(conf.getMetadataStoreOperationTimeoutSeconds(), TimeUnit.SECONDS);
this.state = State.Registered;
} catch (InterruptedException | ExecutionException | TimeoutException e) {
Expand All @@ -135,7 +132,7 @@ public synchronized void register() throws MetadataStoreException {
public synchronized void unregister() throws MetadataStoreException {
if (this.state == State.Registered) {
try {
this.brokerLookupDataLock.release()
brokerLookupDataMetadataCache.delete(brokerIdKeyPath)
.get(conf.getMetadataStoreOperationTimeoutSeconds(), TimeUnit.SECONDS);
this.state = State.Started;
} catch (CompletionException | InterruptedException | ExecutionException | TimeoutException e) {
Expand All @@ -146,19 +143,19 @@ public synchronized void unregister() throws MetadataStoreException {

@Override
public String getBrokerId() {
return this.brokerId;
return pulsar.getBrokerId();
}

@Override
public CompletableFuture<List<String>> getAvailableBrokersAsync() {
this.checkState();
return brokerLookupDataLockManager.listLocks(LOADBALANCE_BROKERS_ROOT).thenApply(ArrayList::new);
return brokerLookupDataMetadataCache.getChildren(LOADBALANCE_BROKERS_ROOT);
}

@Override
public CompletableFuture<Optional<BrokerLookupData>> lookupAsync(String broker) {
this.checkState();
return brokerLookupDataLockManager.readLock(keyPath(broker));
return brokerLookupDataMetadataCache.get(keyPath(broker));
}

public CompletableFuture<Map<String, BrokerLookupData>> getAvailableBrokerLookupDataAsync() {
Expand Down Expand Up @@ -192,7 +189,6 @@ public synchronized void close() throws PulsarServerException {
try {
this.listeners.clear();
this.unregister();
this.brokerLookupDataLockManager.close();
} catch (Exception ex) {
if (ex.getCause() instanceof MetadataStoreException.NotFoundException) {
throw new PulsarServerException.NotFoundException(MetadataStoreException.unwrap(ex));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -128,6 +128,21 @@ public interface MetadataCache<T> {
*/
CompletableFuture<Void> create(String path, T value);

/**
* Create or update the value of the given path in the metadata store without version compare.
* <p>
* This method is equivalent with {@link MetadataStore#put(String, byte[], Optional)} with `Optional.empty()` as
* the 3rd argument. It means if the path does not exist, it will be created. If the path already exists, the new
* value will override the old value.
* </p>
* @param path the path of the object in the metadata store
* @param value the object to put in the metadata store
* @return the future that indicates if this operation failed, it could fail with
* {@link java.io.IOException} if the value failed to be serialized
* {@link MetadataStoreException} if the metadata store operation failed
*/
CompletableFuture<Void> put(String path, T value);

/**
* Delete an object from the metadata store.
* <p>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -243,6 +243,17 @@ public CompletableFuture<Void> create(String path, T value) {
return future;
}

@Override
public CompletableFuture<Void> put(String path, T value) {
final byte[] bytes;
try {
bytes = serde.serialize(path, value);
} catch (IOException e) {
return CompletableFuture.failedFuture(e);
}
return store.put(path, bytes, Optional.empty()).thenAccept(__ -> {});
BewareMyPower marked this conversation as resolved.
Show resolved Hide resolved
}

@Override
public CompletableFuture<Void> delete(String path) {
return store.delete(path, Optional.empty());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -597,4 +597,27 @@ public CustomClass deserialize(String path, byte[] content, Stat stat) throws IO
assertEquals(res.getValue().b, 2);
assertEquals(res.getValue().path, key1);
}

@Test(dataProvider = "distributedImpl")
public void testPut(String provider, Supplier<String> urlSupplier) throws Exception {
@Cleanup
final var store1 = MetadataStoreFactory.create(urlSupplier.get(), MetadataStoreConfig.builder().build());
final var cache1 = store1.getMetadataCache(Integer.class);
@Cleanup
final var store2 = MetadataStoreFactory.create(urlSupplier.get(), MetadataStoreConfig.builder().build());
final var cache2 = store2.getMetadataCache(Integer.class);
final var key = "/testPut";

cache1.put(key, 1); // create
Awaitility.await().untilAsserted(() -> {
assertEquals(cache1.get(key).get().orElse(-1), 1);
assertEquals(cache2.get(key).get().orElse(-1), 1);
});

cache2.put(key, 2); // update
Awaitility.await().untilAsserted(() -> {
assertEquals(cache1.get(key).get().orElse(-1), 2);
assertEquals(cache2.get(key).get().orElse(-1), 2);
});
}
}
Loading