Skip to content

Commit

Permalink
Add extensible load manager impl
Browse files Browse the repository at this point in the history
  • Loading branch information
Demogorgon314 committed Dec 29, 2022
1 parent 89e1fa6 commit acc4788
Show file tree
Hide file tree
Showing 14 changed files with 834 additions and 55 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -22,8 +22,10 @@
import static org.apache.commons.lang3.StringUtils.isBlank;
import static org.apache.commons.lang3.StringUtils.isNotBlank;
import static org.apache.pulsar.broker.resourcegroup.ResourceUsageTransportManager.DISABLE_RESOURCE_USAGE_TRANSPORT_MANAGER;
import static org.apache.pulsar.common.naming.NamespaceName.SYSTEM_NAMESPACE;
import static org.apache.pulsar.common.naming.SystemTopicNames.isTransactionInternalName;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.collect.Sets;
import io.netty.channel.ChannelInitializer;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.socket.SocketChannel;
Expand Down Expand Up @@ -87,14 +89,17 @@
import org.apache.pulsar.broker.loadbalance.LoadReportUpdaterTask;
import org.apache.pulsar.broker.loadbalance.LoadResourceQuotaUpdaterTask;
import org.apache.pulsar.broker.loadbalance.LoadSheddingTask;
import org.apache.pulsar.broker.loadbalance.extensions.ExtensibleLoadManagerWrapper;
import org.apache.pulsar.broker.lookup.v1.TopicLookup;
import org.apache.pulsar.broker.namespace.NamespaceService;
import org.apache.pulsar.broker.protocol.ProtocolHandlers;
import org.apache.pulsar.broker.resourcegroup.ResourceGroupService;
import org.apache.pulsar.broker.resourcegroup.ResourceUsageTopicTransportManager;
import org.apache.pulsar.broker.resourcegroup.ResourceUsageTransportManager;
import org.apache.pulsar.broker.resources.ClusterResources;
import org.apache.pulsar.broker.resources.NamespaceResources;
import org.apache.pulsar.broker.resources.PulsarResources;
import org.apache.pulsar.broker.resources.TenantResources;
import org.apache.pulsar.broker.rest.Topics;
import org.apache.pulsar.broker.service.BrokerService;
import org.apache.pulsar.broker.service.PulsarMetadataEventSynchronizer;
Expand Down Expand Up @@ -137,8 +142,11 @@
import org.apache.pulsar.common.naming.NamespaceBundle;
import org.apache.pulsar.common.naming.NamespaceName;
import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.policies.data.ClusterData;
import org.apache.pulsar.common.policies.data.ClusterDataImpl;
import org.apache.pulsar.common.policies.data.OffloadPoliciesImpl;
import org.apache.pulsar.common.policies.data.Policies;
import org.apache.pulsar.common.policies.data.TenantInfo;
import org.apache.pulsar.common.protocol.schema.SchemaStorage;
import org.apache.pulsar.common.util.FutureUtil;
import org.apache.pulsar.common.util.GracefulExecutorServicesShutdown;
Expand Down Expand Up @@ -794,6 +802,12 @@ public void start() throws PulsarServerException {
}
brokerService.start();

if (this.loadManager.get() instanceof ExtensibleLoadManagerWrapper) {
// Init system namespace for extensible load manager
this.createNamespaceIfNotExists(this.getConfiguration().getClusterName(),
SYSTEM_NAMESPACE.getTenant(), SYSTEM_NAMESPACE);
}

// Load additional servlets
this.brokerAdditionalServlets = AdditionalServlets.load(config);

Expand Down Expand Up @@ -825,6 +839,11 @@ public void start() throws PulsarServerException {
this.webSocketService.setLocalCluster(clusterData);
}

// By starting the Load manager service, the broker will also become visible
// to the rest of the broker by creating the registration z-node. This needs
// to be done only when the broker is fully operative.
this.startLoadManagementService();

// Initialize namespace service, after service url assigned. Should init zk and refresh self owner info.
this.nsService.initialize();

Expand Down Expand Up @@ -866,11 +885,6 @@ public void start() throws PulsarServerException {

this.metricsGenerator = new MetricsGenerator(this);

// By starting the Load manager service, the broker will also become visible
// to the rest of the broker by creating the registration z-node. This needs
// to be done only when the broker is fully operative.
this.startLoadManagementService();

// Initialize the message protocol handlers.
// start the protocol handlers only after the broker is ready,
// so that the protocol handlers can access broker service properly.
Expand Down Expand Up @@ -925,6 +939,36 @@ public void start() throws PulsarServerException {
}
}

protected void createNamespaceIfNotExists(String cluster, String publicTenant, NamespaceName ns) throws Exception {
ClusterResources cr = this.getPulsarResources().getClusterResources();
TenantResources tr = this.getPulsarResources().getTenantResources();
NamespaceResources nsr = this.getPulsarResources().getNamespaceResources();

if (!cr.clusterExists(cluster)) {
cr.createCluster(cluster,
ClusterData.builder()
.serviceUrl(this.getWebServiceAddress())
.serviceUrlTls(this.getWebServiceAddressTls())
.brokerServiceUrl(this.getBrokerServiceUrl())
.brokerServiceUrlTls(this.getBrokerServiceUrlTls())
.build());
}

if (!tr.tenantExists(publicTenant)) {
tr.createTenant(publicTenant,
TenantInfo.builder()
.adminRoles(Sets.newHashSet(config.getSuperUserRoles()))
.allowedClusters(Sets.newHashSet(cluster))
.build());
}

if (!nsr.namespaceExists(ns)) {
Policies nsp = new Policies();
nsp.replication_clusters = Collections.singleton(config.getClusterName());
nsr.createPolicies(ns, nsp);
}
}

private synchronized void createMetricsServlet() {
this.metricsServlet = new PulsarPrometheusMetricsServlet(
this, config.isExposeTopicLevelMetricsInPrometheus(),
Expand Down Expand Up @@ -1085,6 +1129,9 @@ protected void closeLocalMetadataStore() throws Exception {
}

protected void startLeaderElectionService() {
if (this.loadManager.get() instanceof ExtensibleLoadManagerWrapper) {
return;
}
this.leaderElectionService = new LeaderElectionService(coordinationService, getSafeWebServiceAddress(),
state -> {
if (state == LeaderElectionState.Leading) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,8 +25,11 @@
import org.apache.pulsar.broker.PulsarServerException;
import org.apache.pulsar.broker.PulsarService;
import org.apache.pulsar.broker.ServiceConfiguration;
import org.apache.pulsar.broker.loadbalance.extensions.ExtensibleLoadManagerImpl;
import org.apache.pulsar.broker.loadbalance.extensions.ExtensibleLoadManagerWrapper;
import org.apache.pulsar.broker.loadbalance.impl.ModularLoadManagerWrapper;
import org.apache.pulsar.broker.loadbalance.impl.SimpleLoadManagerImpl;
import org.apache.pulsar.broker.lookup.LookupResult;
import org.apache.pulsar.common.naming.ServiceUnitId;
import org.apache.pulsar.common.stats.Metrics;
import org.apache.pulsar.common.util.Reflections;
Expand Down Expand Up @@ -58,6 +61,15 @@ public interface LoadManager {
*/
Optional<ResourceUnit> getLeastLoaded(ServiceUnitId su) throws Exception;

default CompletableFuture<Optional<LookupResult>> findBrokerServiceUrl(
Optional<ServiceUnitId> topic, ServiceUnitId bundle) {
return null;
}

default CompletableFuture<Boolean> checkOwnershipAsync(Optional<ServiceUnitId> topic, ServiceUnitId bundle) {
return null;
}

/**
* Generate the load report.
*/
Expand Down Expand Up @@ -143,6 +155,11 @@ static LoadManager create(final PulsarService pulsar) {
final LoadManager casted = new ModularLoadManagerWrapper((ModularLoadManager) loadManagerInstance);
casted.initialize(pulsar);
return casted;
} else if (loadManagerInstance instanceof ExtensibleLoadManagerImpl) {
final LoadManager casted =
new ExtensibleLoadManagerWrapper((ExtensibleLoadManagerImpl) loadManagerInstance);
casted.initialize(pulsar);
return casted;
}
} catch (Exception e) {
LOG.warn("Error when trying to create load manager: ", e);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -82,7 +82,7 @@ public BrokerRegistryImpl(PulsarService pulsar) {
this.listeners = new ArrayList<>();
this.brokerId = pulsar.getLookupServiceAddress();
this.brokerLookupData = new BrokerLookupData(
pulsar.getSafeWebServiceAddress(),
pulsar.getWebServiceAddress(),
pulsar.getWebServiceAddressTls(),
pulsar.getBrokerServiceUrl(),
pulsar.getBrokerServiceUrlTls(),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -64,6 +64,15 @@ public interface ExtensibleLoadManager extends Closeable {
*/
CompletableFuture<Optional<BrokerLookupData>> assign(Optional<ServiceUnitId> topic, ServiceUnitId serviceUnit);

/**
* Check the incoming service unit is owned by the current broker.
*
* @param topic The optional topic, some method won't provide topic var in this param.
* @param serviceUnit The service unit (e.g. bundle).
* @return The broker lookup data.
*/
CompletableFuture<Boolean> checkOwnershipAsync(Optional<ServiceUnitId> topic, ServiceUnitId serviceUnit);

/**
* Close the load manager.
*
Expand Down
Loading

0 comments on commit acc4788

Please sign in to comment.