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] PIP-192: Make unload and transfer admin API functional #19538

Original file line number Diff line number Diff line change
Expand Up @@ -58,6 +58,7 @@
import org.apache.pulsar.broker.admin.AdminResource;
import org.apache.pulsar.broker.authorization.AuthorizationService;
import org.apache.pulsar.broker.loadbalance.LeaderBroker;
import org.apache.pulsar.broker.loadbalance.extensions.ExtensibleLoadManagerImpl;
import org.apache.pulsar.broker.service.BrokerServiceException;
import org.apache.pulsar.broker.service.BrokerServiceException.SubscriptionBusyException;
import org.apache.pulsar.broker.service.Subscription;
Expand Down Expand Up @@ -983,8 +984,17 @@ public CompletableFuture<Void> setNamespaceBundleAffinityAsync(String bundleRang
}
return CompletableFuture.completedFuture(null);
})
.thenCompose(__ -> validateLeaderBrokerAsync())
.thenCompose(__ -> {
if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(config())) {
return CompletableFuture.completedFuture(null);
}
return validateLeaderBrokerAsync();
})
.thenAccept(__ -> {
if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(config())) {
return;
}
// For ExtensibleLoadManager, this operation will be ignored.
pulsar().getLoadManager().get().setNamespaceBundleAffinity(bundleRange, destinationBroker);
});
}
Expand Down Expand Up @@ -1036,10 +1046,11 @@ public CompletableFuture<Void> internalUnloadNamespaceBundleAsync(String bundleR
namespaceName, bundleRange);
return CompletableFuture.completedFuture(null);
}
Optional<String> destinationBrokerOpt = Optional.ofNullable(destinationBroker);
return validateNamespaceBundleOwnershipAsync(namespaceName, policies.bundles, bundleRange,
authoritative, true)
.thenCompose(nsBundle ->
pulsar().getNamespaceService().unloadNamespaceBundle(nsBundle));
.thenCompose(nsBundle -> pulsar().getNamespaceService()
.unloadNamespaceBundle(nsBundle, destinationBrokerOpt));
}));
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,7 @@
import org.apache.pulsar.broker.PulsarService;
import org.apache.pulsar.broker.ServiceConfiguration;
import org.apache.pulsar.broker.loadbalance.BrokerFilterException;
import org.apache.pulsar.broker.loadbalance.LoadManager;
import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannel;
import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl;
import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLoadData;
Expand All @@ -43,9 +44,11 @@
import org.apache.pulsar.broker.loadbalance.extensions.filter.BrokerFilter;
import org.apache.pulsar.broker.loadbalance.extensions.filter.BrokerMaxTopicCountFilter;
import org.apache.pulsar.broker.loadbalance.extensions.filter.BrokerVersionFilter;
import org.apache.pulsar.broker.loadbalance.extensions.manager.UnloadManager;
import org.apache.pulsar.broker.loadbalance.extensions.models.AssignCounter;
import org.apache.pulsar.broker.loadbalance.extensions.models.SplitCounter;
import org.apache.pulsar.broker.loadbalance.extensions.models.SplitDecision;
import org.apache.pulsar.broker.loadbalance.extensions.models.Unload;
import org.apache.pulsar.broker.loadbalance.extensions.models.UnloadCounter;
import org.apache.pulsar.broker.loadbalance.extensions.models.UnloadDecision;
import org.apache.pulsar.broker.loadbalance.extensions.reporter.BrokerLoadDataReporter;
Expand Down Expand Up @@ -110,6 +113,8 @@ public class ExtensibleLoadManagerImpl implements ExtensibleLoadManager {
private ScheduledFuture brokerLoadDataReportTask;
private ScheduledFuture topBundlesLoadDataReportTask;

private UnloadManager unloadManager;

private boolean started = false;

private final AssignCounter assignCounter = new AssignCounter();
Expand Down Expand Up @@ -143,6 +148,13 @@ public static boolean isLoadManagerExtensionEnabled(ServiceConfiguration conf) {
return ExtensibleLoadManagerImpl.class.getName().equals(conf.getLoadManagerClassName());
}

public static ExtensibleLoadManagerImpl get(LoadManager loadManager) {
if (!(loadManager instanceof ExtensibleLoadManagerWrapper loadManagerWrapper)) {
throw new IllegalArgumentException("The load manager should be 'ExtensibleLoadManagerWrapper'.");
}
return loadManagerWrapper.get();
}

@Override
public void start() throws PulsarServerException {
if (this.started) {
Expand All @@ -151,6 +163,8 @@ public void start() throws PulsarServerException {
this.brokerRegistry = new BrokerRegistryImpl(pulsar);
this.serviceUnitStateChannel = new ServiceUnitStateChannelImpl(pulsar);
this.brokerRegistry.start();
this.unloadManager = new UnloadManager();
this.serviceUnitStateChannel.listen(unloadManager);
this.serviceUnitStateChannel.start();

try {
Expand Down Expand Up @@ -201,7 +215,8 @@ public void start() throws PulsarServerException {
interval, TimeUnit.MILLISECONDS);

// TODO: Start bundle split scheduler.
this.unloadScheduler = new UnloadScheduler(pulsar.getLoadManagerExecutor(), context, serviceUnitStateChannel);
this.unloadScheduler = new UnloadScheduler(
pulsar.getLoadManagerExecutor(), unloadManager, context, serviceUnitStateChannel);
this.unloadScheduler.start();
this.started = true;
}
Expand Down Expand Up @@ -300,15 +315,48 @@ private CompletableFuture<Optional<String>> selectAsync(ServiceUnitId bundle) {

@Override
public CompletableFuture<Boolean> checkOwnershipAsync(Optional<ServiceUnitId> topic, ServiceUnitId bundleUnit) {
return getOwnershipAsync(topic, bundleUnit)
.thenApply(broker -> brokerRegistry.getBrokerId().equals(broker.orElse(null)));
}

private CompletableFuture<Optional<String>> getOwnershipAsync(Optional<ServiceUnitId> topic,
ServiceUnitId bundleUnit) {
final String bundle = bundleUnit.toString();
CompletableFuture<Optional<String>> owner;
if (topic.isPresent() && isInternalTopic(topic.get().toString())) {
owner = serviceUnitStateChannel.getChannelOwnerAsync();
} else {
owner = serviceUnitStateChannel.getOwnerAsync(bundle);
}
return owner;
}

public CompletableFuture<Void> unloadNamespaceBundleAsync(ServiceUnitId bundle,
Optional<String> destinationBroker) {
return getOwnershipAsync(Optional.empty(), bundle)
.thenCompose(brokerOpt -> {
if (brokerOpt.isEmpty()) {
String msg = String.format("Namespace bundle: %s is not owned by any broker.", bundle);
log.warn(msg);
throw new IllegalStateException(msg);
}
String sourceBroker = brokerOpt.get();
if (destinationBroker.isPresent() && sourceBroker.endsWith(destinationBroker.get())) {
String msg = String.format("Namespace bundle: %s own by %s, cannot be transfer to same broker.",
bundle, sourceBroker);
log.warn(msg);
throw new IllegalArgumentException(msg);
}
return unloadAsync(new Unload(sourceBroker, bundle.toString(), destinationBroker),
conf.getNamespaceBundleUnloadingTimeoutMs(), TimeUnit.MILLISECONDS);
});
}

return owner.thenApply(broker -> brokerRegistry.getBrokerId().equals(broker.orElse(null)));
private CompletableFuture<Void> unloadAsync(Unload unload,
long timeout,
TimeUnit timeoutUnit) {
CompletableFuture<Void> future = serviceUnitStateChannel.publishUnloadEventAsync(unload);
return unloadManager.waitAsync(future, unload.serviceUnit(), timeout, timeoutUnit);
}

@Override
Expand Down Expand Up @@ -337,6 +385,7 @@ public void close() throws PulsarServerException {
try {
this.serviceUnitStateChannel.close();
} finally {
this.unloadManager.close();
this.started = false;
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@
import java.util.Optional;
import java.util.concurrent.CompletableFuture;
import org.apache.pulsar.broker.PulsarServerException;
import org.apache.pulsar.broker.loadbalance.extensions.manager.StateChangeListener;
import org.apache.pulsar.broker.loadbalance.extensions.models.Split;
import org.apache.pulsar.broker.loadbalance.extensions.models.Unload;
import org.apache.pulsar.common.stats.Metrics;
Expand Down Expand Up @@ -156,4 +157,11 @@ public interface ServiceUnitStateChannel extends Closeable {
*/
List<Metrics> getMetrics();

/**
* Add a state change listener.
*
* @param listener State change listener.
*/
void listen(StateChangeListener listener);

}
Original file line number Diff line number Diff line change
Expand Up @@ -72,6 +72,7 @@
import org.apache.pulsar.broker.loadbalance.extensions.BrokerRegistry;
import org.apache.pulsar.broker.loadbalance.extensions.ExtensibleLoadManagerWrapper;
import org.apache.pulsar.broker.loadbalance.extensions.LoadManagerContext;
import org.apache.pulsar.broker.loadbalance.extensions.manager.StateChangeListener;
import org.apache.pulsar.broker.loadbalance.extensions.models.Split;
import org.apache.pulsar.broker.loadbalance.extensions.models.Unload;
import org.apache.pulsar.broker.loadbalance.extensions.strategy.BrokerSelectionStrategy;
Expand Down Expand Up @@ -117,6 +118,7 @@ public class ServiceUnitStateChannelImpl implements ServiceUnitStateChannel {
private final ConcurrentOpenHashMap<String, CompletableFuture<String>> getOwnerRequests;
private final String lookupServiceAddress;
private final ConcurrentOpenHashMap<String, CompletableFuture<Void>> cleanupJobs;
private final StateChangeListeners stateChangeListeners;
private final LeaderElectionService leaderElectionService;
private BrokerSelectionStrategy brokerSelector;
private BrokerRegistry brokerRegistry;
Expand Down Expand Up @@ -191,6 +193,7 @@ public ServiceUnitStateChannelImpl(PulsarService pulsar) {
this.getOwnerRequests = ConcurrentOpenHashMap.<String,
CompletableFuture<String>>newBuilder().build();
this.cleanupJobs = ConcurrentOpenHashMap.<String, CompletableFuture<Void>>newBuilder().build();
this.stateChangeListeners = new StateChangeListeners();
this.semiTerminalStateWaitingTimeInMillis = config.getLoadBalancerServiceUnitStateCleanUpDelayTimeInSeconds()
* 1000;
this.inFlightStateWaitingTimeInMillis = MAX_IN_FLIGHT_STATE_WAITING_TIME_IN_MILLIS;
Expand Down Expand Up @@ -357,6 +360,10 @@ public synchronized void close() throws PulsarServerException {
log.info("Successfully cancelled the cleanup tasks");
}

if (stateChangeListeners != null) {
stateChangeListeners.close();
}

log.info("Successfully closed the channel.");

} catch (Exception e) {
Expand Down Expand Up @@ -619,6 +626,7 @@ private void handleOwnEvent(String serviceUnit, ServiceUnitStateData data) {
if (getOwnerRequest != null) {
getOwnerRequest.complete(data.dstBroker());
}
stateChangeListeners.notify(serviceUnit, data, null);
if (isTargetBroker(data.dstBroker())) {
log(null, serviceUnit, data, null);
}
Expand All @@ -628,7 +636,7 @@ private void handleAssignEvent(String serviceUnit, ServiceUnitStateData data) {
if (isTargetBroker(data.dstBroker())) {
ServiceUnitStateData next = new ServiceUnitStateData(
Owned, data.dstBroker(), data.sourceBroker(), getNextVersionId(data));
pubAsync(serviceUnit, next)
stateChangeListeners.notifyOnCompletion(pubAsync(serviceUnit, next), serviceUnit, data)
.whenComplete((__, e) -> log(e, serviceUnit, data, next));
}
}
Expand All @@ -644,15 +652,15 @@ private void handleReleaseEvent(String serviceUnit, ServiceUnitStateData data) {
next = new ServiceUnitStateData(
Free, null, data.sourceBroker(), getNextVersionId(data));
}
closeServiceUnit(serviceUnit)
.thenCompose(__ -> pubAsync(serviceUnit, next))
stateChangeListeners.notifyOnCompletion(closeServiceUnit(serviceUnit)
.thenCompose(__ -> pubAsync(serviceUnit, next)), serviceUnit, data)
.whenComplete((__, e) -> log(e, serviceUnit, data, next));
}
}

private void handleSplitEvent(String serviceUnit, ServiceUnitStateData data) {
if (isTargetBroker(data.sourceBroker())) {
splitServiceUnit(serviceUnit, data)
stateChangeListeners.notifyOnCompletion(splitServiceUnit(serviceUnit, data), serviceUnit, data)
.whenComplete((__, e) -> log(e, serviceUnit, data, null));
}
}
Expand All @@ -662,6 +670,7 @@ private void handleFreeEvent(String serviceUnit, ServiceUnitStateData data) {
if (getOwnerRequest != null) {
getOwnerRequest.complete(null);
}
stateChangeListeners.notify(serviceUnit, data, null);
if (isTargetBroker(data.sourceBroker())) {
log(null, serviceUnit, data, null);
}
Expand All @@ -672,6 +681,7 @@ private void handleDeleteEvent(String serviceUnit, ServiceUnitStateData data) {
if (getOwnerRequest != null) {
getOwnerRequest.completeExceptionally(new IllegalStateException(serviceUnit + "has been deleted."));
}
stateChangeListeners.notify(serviceUnit, data, null);
if (isTargetBroker(data.sourceBroker())) {
log(null, serviceUnit, data, null);
}
Expand All @@ -682,6 +692,7 @@ private void handleInitEvent(String serviceUnit) {
if (getOwnerRequest != null) {
getOwnerRequest.complete(null);
}
stateChangeListeners.notify(serviceUnit, null, null);
log(null, serviceUnit, null, null);
}

Expand Down Expand Up @@ -1302,4 +1313,9 @@ public List<Metrics> getMetrics() {

return metrics;
}

@Override
public void listen(StateChangeListener listener) {
this.stateChangeListeners.addListener(listener);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,67 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.pulsar.broker.loadbalance.extensions.channel;

import java.util.List;
import java.util.Objects;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CopyOnWriteArrayList;
import lombok.extern.slf4j.Slf4j;
import org.apache.pulsar.broker.loadbalance.extensions.manager.StateChangeListener;

@Slf4j
public class StateChangeListeners {

private final List<StateChangeListener> stateChangeListeners;

public StateChangeListeners() {
stateChangeListeners = new CopyOnWriteArrayList<>();
}

public void addListener(StateChangeListener listener) {
Objects.requireNonNull(listener);
stateChangeListeners.add(listener);
}

public void close() {
this.stateChangeListeners.clear();
}

/**
* Notify all currently added listeners on completion of the future.
*
* @return future of a new completion stage
*/
public <T> CompletableFuture<T> notifyOnCompletion(CompletableFuture<T> future,
String serviceUnit,
ServiceUnitStateData data) {
return future.whenComplete((r, ex) -> notify(serviceUnit, data, ex));
}

public void notify(String serviceUnit, ServiceUnitStateData data, Throwable t) {
stateChangeListeners.forEach(listener -> {
try {
listener.handleEvent(serviceUnit, data, t);
} catch (Throwable ex) {
log.error("StateChangeListener: {} exception while handling {} for service unit {}",
listener, data, serviceUnit, ex);
}
});
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,33 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.pulsar.broker.loadbalance.extensions.manager;

import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateData;

public interface StateChangeListener {

/**
* Handle the service unit state change.
*
* @param serviceUnit - Service Unit(Namespace bundle).
* @param data - Service unit state data.
* @param t - Exception, if present.
*/
void handleEvent(String serviceUnit, ServiceUnitStateData data, Throwable t);
}
Loading