Skip to content

Commit

Permalink
Added SplitManager
Browse files Browse the repository at this point in the history
  • Loading branch information
heesung-sn committed Mar 8, 2023
1 parent 42c23e6 commit c856344
Show file tree
Hide file tree
Showing 7 changed files with 368 additions and 30 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -44,6 +44,7 @@
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.SplitManager;
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;
Expand Down Expand Up @@ -115,6 +116,8 @@ public class ExtensibleLoadManagerImpl implements ExtensibleLoadManager {

private UnloadManager unloadManager;

private SplitManager splitManager;

private boolean started = false;

private final AssignCounter assignCounter = new AssignCounter();
Expand Down Expand Up @@ -164,7 +167,9 @@ public void start() throws PulsarServerException {
this.serviceUnitStateChannel = new ServiceUnitStateChannelImpl(pulsar);
this.brokerRegistry.start();
this.unloadManager = new UnloadManager();
this.splitManager = new SplitManager(splitCounter);
this.serviceUnitStateChannel.listen(unloadManager);
this.serviceUnitStateChannel.listen(splitManager);
this.serviceUnitStateChannel.start();

try {
Expand Down Expand Up @@ -217,7 +222,7 @@ public void start() throws PulsarServerException {
pulsar.getLoadManagerExecutor(), unloadManager, context, serviceUnitStateChannel);
this.unloadScheduler.start();
this.splitScheduler = new SplitScheduler(
pulsar, serviceUnitStateChannel, splitCounter, splitMetrics, context);
pulsar, serviceUnitStateChannel, splitManager, splitCounter, splitMetrics, context);
this.splitScheduler.start();
this.started = true;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -882,6 +882,7 @@ protected void splitServiceUnitOnceAndRetry(NamespaceService namespaceService,
return null;
});
}

public void handleMetadataSessionEvent(SessionEvent e) {
if (e == SessionReestablished || e == SessionLost) {
lastMetadataSessionEvent = e;
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,121 @@
/*
* 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 static org.apache.pulsar.broker.loadbalance.extensions.models.SplitDecision.Label.Failure;
import static org.apache.pulsar.broker.loadbalance.extensions.models.SplitDecision.Reason.Unknown;
import java.util.Map;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.TimeUnit;
import lombok.extern.slf4j.Slf4j;
import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState;
import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateData;
import org.apache.pulsar.broker.loadbalance.extensions.models.SplitCounter;
import org.apache.pulsar.broker.loadbalance.extensions.models.SplitDecision;

/**
* Split manager.
*/
@Slf4j
public class SplitManager implements StateChangeListener {

record InFlightSplitRequest(SplitDecision splitDecision, CompletableFuture<Void> future) {
}

private final Map<String, InFlightSplitRequest> inFlightSplitRequests;

private final SplitCounter counter;

public SplitManager(SplitCounter splitCounter) {
this.inFlightSplitRequests = new ConcurrentHashMap<>();
this.counter = splitCounter;
}

private void complete(String serviceUnit, Throwable ex) {
inFlightSplitRequests.computeIfPresent(serviceUnit, (__, inFlightSplitRequest) -> {
var future = inFlightSplitRequest.future;
if (!future.isDone()) {
if (ex != null) {
counter.update(Failure, Unknown);
future.completeExceptionally(ex);
if (log.isDebugEnabled()) {
log.debug("Complete exceptionally split bundle: {}", serviceUnit, ex);
}
} else {
counter.update(inFlightSplitRequest.splitDecision);
future.complete(null);
if (log.isDebugEnabled()) {
log.debug("Complete split bundle: {}", serviceUnit);
}
}
}
return null;
});
}

public CompletableFuture<Void> waitAsync(CompletableFuture<Void> eventPubFuture,
String bundle,
SplitDecision decision,
long timeout,
TimeUnit timeoutUnit) {

return eventPubFuture.thenCompose(__ -> inFlightSplitRequests.computeIfAbsent(bundle, ignore -> {
if (log.isDebugEnabled()) {
log.debug("Handle split bundle: {}, timeout: {} {}", bundle, timeout, timeoutUnit);
}
CompletableFuture<Void> future = new CompletableFuture<>();
future.orTimeout(timeout, timeoutUnit).whenComplete((v, ex) -> {
if (ex != null) {
inFlightSplitRequests.remove(bundle);
log.warn("Failed to wait for split for serviceUnit: {}", bundle, ex);
}
});
return new InFlightSplitRequest(decision, future);
}).future);
}

@Override
public void handleEvent(String serviceUnit, ServiceUnitStateData data, Throwable t) {
ServiceUnitState state = ServiceUnitStateData.state(data);
if (t != null && inFlightSplitRequests.containsKey(serviceUnit)) {
this.complete(serviceUnit, t);
return;
}
switch (state) {
case Deleted, Owned, Init -> this.complete(serviceUnit, t);
default -> {
if (log.isDebugEnabled()) {
log.debug("Handling {} for service unit {}", data, serviceUnit);
}
}
}
}

public void close() {
inFlightSplitRequests.forEach((bundle, inFlightSplitRequest) -> {
if (!inFlightSplitRequest.future.isDone()) {
String msg = String.format("Splitting bundle: %s, but the manager already closed.", bundle);
log.warn(msg);
inFlightSplitRequest.future.completeExceptionally(new IllegalStateException(msg));
}
});
inFlightSplitRequests.clear();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,7 @@
import org.apache.pulsar.broker.ServiceConfiguration;
import org.apache.pulsar.broker.loadbalance.extensions.LoadManagerContext;
import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannel;
import org.apache.pulsar.broker.loadbalance.extensions.manager.SplitManager;
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.strategy.DefaultNamespaceBundleSplitStrategyImpl;
Expand Down Expand Up @@ -61,6 +62,8 @@ public class SplitScheduler implements LoadManagerScheduler {

private final SplitCounter counter;

private final SplitManager splitManager;

private final AtomicReference<List<Metrics>> splitMetrics;

private volatile ScheduledFuture<?> task;
Expand All @@ -69,12 +72,14 @@ public class SplitScheduler implements LoadManagerScheduler {

public SplitScheduler(PulsarService pulsar,
ServiceUnitStateChannel serviceUnitStateChannel,
SplitManager splitManager,
SplitCounter counter,
AtomicReference<List<Metrics>> splitMetrics,
LoadManagerContext context,
NamespaceBundleSplitStrategy bundleSplitStrategy) {
this.pulsar = pulsar;
this.loadManagerExecutor = pulsar.getLoadManagerExecutor();
this.splitManager = splitManager;
this.counter = counter;
this.splitMetrics = splitMetrics;
this.context = context;
Expand All @@ -85,10 +90,11 @@ public SplitScheduler(PulsarService pulsar,

public SplitScheduler(PulsarService pulsar,
ServiceUnitStateChannel serviceUnitStateChannel,
SplitManager splitManager,
SplitCounter counter,
AtomicReference<List<Metrics>> splitMetrics,
LoadManagerContext context) {
this(pulsar, serviceUnitStateChannel, counter, splitMetrics, context,
this(pulsar, serviceUnitStateChannel, splitManager, counter, splitMetrics, context,
new DefaultNamespaceBundleSplitStrategyImpl(counter));
}

Expand All @@ -110,27 +116,36 @@ public void execute() {
synchronized (bundleSplitStrategy) {
final Set<SplitDecision> decisions = bundleSplitStrategy.findBundlesToSplit(context, pulsar);
if (!decisions.isEmpty()) {

// currently following the unloading timeout
var asyncOpTimeoutMs = conf.getNamespaceBundleUnloadingTimeoutMs();
List<CompletableFuture<Void>> futures = new ArrayList<>();
for (SplitDecision decision : decisions) {
if (decision.getLabel() == Success) {
var split = decision.getSplit();
futures.add(serviceUnitStateChannel.publishSplitEventAsync(split)
.whenComplete((__, e) -> {
if (e == null) {
counter.update(decision);
log.info("Published Split Event for {}", split);
} else {
counter.update(Failure, Unknown);
log.error("Failed to publish Split Event for {}", split);
}
}));
futures.add(
splitManager.waitAsync(
serviceUnitStateChannel.publishSplitEventAsync(split)
.whenComplete((__, e) -> {
if (e == null) {
log.info("Published Split Event for {}", split);
} else {
counter.update(Failure, Unknown);
log.error("Failed to publish Split Event for {}", split);
}
}),
split.serviceUnit(),
decision,
asyncOpTimeoutMs, TimeUnit.MILLISECONDS)
);
}
}
FutureUtil.waitForAll(futures).exceptionally(ex -> {
log.error("Failed to wait for split events to persist.", ex);
counter.update(Failure, Unknown);
return null;
});
try {
FutureUtil.waitForAll(futures)
.get(asyncOpTimeoutMs, TimeUnit.MILLISECONDS);
} catch (Throwable e) {
log.error("Failed to wait for split events to persist.", e);
}
} else {
if (debugMode) {
log.info("BundleSplitStrategy returned no bundles to split.");
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -79,14 +79,7 @@ public Set<SplitDecision> findBundlesToSplit(LoadManagerContext context, PulsarS
pulsar.getNamespaceService().getNamespaceBundleFactory();

// clean bundleHighTrafficFrequency
var bundleHighTrafficIterator =
bundleHighTrafficFrequency.entrySet().iterator();
while (bundleHighTrafficIterator.hasNext()) {
String bundle = bundleHighTrafficIterator.next().getKey();
if (!bundleStatsMap.containsKey(bundle)) {
bundleHighTrafficIterator.remove();
}
}
bundleHighTrafficFrequency.keySet().retainAll(bundleStatsMap.keySet());

for (var entry : bundleStatsMap.entrySet()) {
final String bundle = entry.getKey();
Expand Down
Loading

0 comments on commit c856344

Please sign in to comment.