Skip to content

Commit

Permalink
[improve][broker] PIP-192 Added ServiceUnitStateCompactionStrategy (#…
Browse files Browse the repository at this point in the history
  • Loading branch information
heesung-sn authored Jan 31, 2023
1 parent 72b2e7e commit 1cd1aef
Show file tree
Hide file tree
Showing 11 changed files with 1,168 additions and 54 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -146,6 +146,7 @@
import org.apache.pulsar.common.util.ThreadDumpUtil;
import org.apache.pulsar.common.util.netty.EventLoopUtil;
import org.apache.pulsar.compaction.Compactor;
import org.apache.pulsar.compaction.StrategicTwoPhaseCompactor;
import org.apache.pulsar.compaction.TwoPhaseCompactor;
import org.apache.pulsar.functions.worker.ErrorNotifier;
import org.apache.pulsar.functions.worker.WorkerConfig;
Expand Down Expand Up @@ -198,6 +199,7 @@ public class PulsarService implements AutoCloseable, ShutdownService {
private TopicPoliciesService topicPoliciesService = TopicPoliciesService.DISABLED;
private BookKeeperClientFactory bkClientFactory;
private Compactor compactor;
private StrategicTwoPhaseCompactor strategicCompactor;
private ResourceUsageTransportManager resourceUsageTransportManager;
private ResourceGroupService resourceGroupServiceManager;

Expand Down Expand Up @@ -1473,6 +1475,19 @@ public Compactor getNullableCompactor() {
return this.compactor;
}

public StrategicTwoPhaseCompactor newStrategicCompactor() throws PulsarServerException {
return new StrategicTwoPhaseCompactor(this.getConfiguration(),
getClient(), getBookKeeperClient(),
getCompactorExecutor());
}

public synchronized StrategicTwoPhaseCompactor getStrategicCompactor() throws PulsarServerException {
if (this.strategicCompactor == null) {
this.strategicCompactor = newStrategicCompactor();
}
return this.strategicCompactor;
}

protected synchronized OrderedScheduler getOffloaderScheduler(OffloadPoliciesImpl offloadPolicies) {
if (this.offloaderScheduler == null) {
this.offloaderScheduler = OrderedScheduler.newSchedulerBuilder()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -66,7 +66,9 @@ public enum ServiceUnitState {
Splitting; // the service unit(e.g. bundle) is in the process of splitting.

private static Map<ServiceUnitState, Set<ServiceUnitState>> validTransitions = Map.of(
Free, Set.of(Owned, Assigned),
// (Free -> Released | Splitting) transitions are required
// when the topic is compacted in the middle of transfer or split.
Free, Set.of(Owned, Assigned, Released, Splitting),
Owned, Set.of(Assigned, Splitting, Free),
Assigned, Set.of(Owned, Released, Free),
Released, Set.of(Owned, Free),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -43,7 +43,7 @@
import java.util.concurrent.ScheduledFuture;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
import lombok.extern.slf4j.Slf4j;
import org.apache.commons.lang3.StringUtils;
import org.apache.commons.lang3.mutable.MutableInt;
Expand Down Expand Up @@ -101,7 +101,7 @@ public class ServiceUnitStateChannelImpl implements ServiceUnitStateChannel {
private long totalCleanupCnt = 0;
private long totalBrokerCleanupTombstoneCnt = 0;
private long totalServiceUnitCleanupTombstoneCnt = 0;
private long totalServiceUnitCleanupErrorCnt = 0;
private AtomicLong totalCleanupErrorCnt = new AtomicLong();
private long totalCleanupScheduledCnt = 0;
private long totalCleanupIgnoredCnt = 0;
private long totalCleanupCancelledCnt = 0;
Expand Down Expand Up @@ -175,10 +175,11 @@ public synchronized void start() throws PulsarServerException {
}
tableview = pulsar.getClient().newTableViewBuilder(schema)
.topic(TOPIC)
// TODO: enable CompactionStrategy
.loadConf(Map.of(
"topicCompactionStrategyClassName",
ServiceUnitStateCompactionStrategy.class.getName()))
.create();
// TODO: schedule listen instead of foreachAndListen
tableview.forEachAndListen((key, value) -> handle(key, value));
tableview.listen((key, value) -> handle(key, value));
log.debug("Successfully started the channel tableview.");

pulsar.getLocalMetadataStore().registerSessionListener(this::handleMetadataSessionEvent);
Expand Down Expand Up @@ -332,8 +333,6 @@ private void handle(String serviceUnit, ServiceUnitStateData data) {
}

ServiceUnitState state = data == null ? Free : data.state();

// TODO : Add state validation in tableview by the compaction strategy
switch (state) {
case Owned -> handleOwnEvent(serviceUnit, data);
case Assigned -> handleAssignEvent(serviceUnit, data);
Expand Down Expand Up @@ -599,7 +598,16 @@ private void scheduleCleanup(String broker, long delayInSecs) {
.delayedExecutor(delayInSecs, TimeUnit.SECONDS, pulsar.getLoadManagerExecutor());
totalCleanupScheduledCnt++;
return CompletableFuture
.runAsync(() -> doCleanup(broker), delayed);
.runAsync(() -> {
try {
doCleanup(broker);
} catch (Throwable e) {
log.error("Failed to run the cleanup job for the broker {}, "
+ "totalCleanupErrorCnt:{}.",
broker, totalCleanupErrorCnt.incrementAndGet(), e);
}
}
, delayed);
});

log.info("Scheduled ownership cleanup for broker:{} with delay:{} secs. Pending clean jobs:{}.",
Expand All @@ -610,23 +618,23 @@ private void scheduleCleanup(String broker, long delayInSecs) {
private void doCleanup(String broker) {
long startTime = System.nanoTime();
log.info("Started ownership cleanup for the inactive broker:{}", broker);
AtomicInteger serviceUnitTombstoneCnt = new AtomicInteger();
AtomicInteger serviceUnitTombstoneErrorCnt = new AtomicInteger();
int serviceUnitTombstoneCnt = 0;
long totalCleanupErrorCntStart = totalCleanupErrorCnt.get();
for (Map.Entry<String, ServiceUnitStateData> etr : tableview.entrySet()) {
ServiceUnitStateData stateData = etr.getValue();
String serviceUnit = etr.getKey();
if (StringUtils.equals(broker, stateData.broker())
|| StringUtils.equals(broker, stateData.sourceBroker())) {
log.info("Cleaning ownership serviceUnit:{}, stateData:{}.", serviceUnit, stateData);
tombstoneAsync(serviceUnit).whenComplete((__, e) -> {
if (e == null) {
serviceUnitTombstoneCnt.incrementAndGet();
} else {
log.error("Failed cleaning the ownership serviceUnit:{}, stateData:{}.",
serviceUnit, stateData);
serviceUnitTombstoneErrorCnt.incrementAndGet();
if (e != null) {
log.error("Failed cleaning the ownership serviceUnit:{}, stateData:{}, "
+ "cleanupErrorCnt:{}.",
serviceUnit, stateData,
totalCleanupErrorCnt.incrementAndGet() - totalCleanupErrorCntStart);
}
});
serviceUnitTombstoneCnt++;
}
}

Expand All @@ -636,26 +644,22 @@ private void doCleanup(String broker) {
log.error("Failed to flush the in-flight messages.", e);
}

if (serviceUnitTombstoneCnt.get() > 0) {
if (serviceUnitTombstoneCnt > 0) {
this.totalCleanupCnt++;
this.totalServiceUnitCleanupTombstoneCnt += serviceUnitTombstoneCnt.get();
this.totalServiceUnitCleanupTombstoneCnt += serviceUnitTombstoneCnt;
this.totalBrokerCleanupTombstoneCnt++;
}

if (serviceUnitTombstoneErrorCnt.get() > 0) {
this.totalServiceUnitCleanupErrorCnt += serviceUnitTombstoneErrorCnt.get();
}

double cleanupTime = TimeUnit.NANOSECONDS
.toMillis((System.nanoTime() - startTime));
// TODO: clean load data stores
log.info("Completed a cleanup for the inactive broker:{} in {} ms. "
+ "Published tombstone for orphan service units: serviceUnitTombstoneCnt:{}, "
+ "serviceUnitTombstoneErrorCnt:{}, metrics:{} ",
+ "approximate cleanupErrorCnt:{}, metrics:{} ",
broker,
cleanupTime,
serviceUnitTombstoneCnt,
serviceUnitTombstoneErrorCnt,
totalCleanupErrorCntStart - totalCleanupErrorCnt.get(),
printCleanupMetrics());
cleanupJobs.remove(broker);
}
Expand All @@ -675,8 +679,8 @@ private void monitorOwnerships(List<String> brokers) {
long startTime = System.nanoTime();
Set<String> inactiveBrokers = new HashSet<>();
Set<String> activeBrokers = new HashSet<>(brokers);
AtomicInteger serviceUnitTombstoneCnt = new AtomicInteger();
AtomicInteger serviceUnitTombstoneErrorCnt = new AtomicInteger();
int serviceUnitTombstoneCnt = 0;
long totalCleanupErrorCntStart = totalCleanupErrorCnt.get();
long now = System.currentTimeMillis();
for (Map.Entry<String, ServiceUnitStateData> etr : tableview.entrySet()) {
String serviceUnit = etr.getKey();
Expand All @@ -690,14 +694,14 @@ private void monitorOwnerships(List<String> brokers) {
serviceUnit, stateData);

tombstoneAsync(serviceUnit).whenComplete((__, e) -> {
if (e == null) {
serviceUnitTombstoneCnt.incrementAndGet();
} else {
log.error("Failed cleaning the ownership serviceUnit:{}, stateData:{}.",
serviceUnit, stateData);
serviceUnitTombstoneErrorCnt.incrementAndGet();
if (e != null) {
log.error("Failed cleaning the ownership serviceUnit:{}, stateData:{}, "
+ "cleanupErrorCnt:{}.",
serviceUnit, stateData,
totalCleanupErrorCnt.incrementAndGet() - totalCleanupErrorCntStart);
}
});
serviceUnitTombstoneCnt++;
}
}

Expand All @@ -711,36 +715,35 @@ private void monitorOwnerships(List<String> brokers) {
log.error("Failed to flush the in-flight messages.", e);
}

if (serviceUnitTombstoneCnt.get() > 0) {
this.totalServiceUnitCleanupTombstoneCnt += serviceUnitTombstoneCnt.get();
if (serviceUnitTombstoneCnt > 0) {
this.totalServiceUnitCleanupTombstoneCnt += serviceUnitTombstoneCnt;
}
this.totalServiceUnitCleanupErrorCnt += serviceUnitTombstoneErrorCnt.get();

double monitorTime = TimeUnit.NANOSECONDS
.toMillis((System.nanoTime() - startTime));
log.info("Completed the ownership monitor run in {} ms. "
+ "Scheduled cleanups for inactiveBrokers:{}. inactiveBrokerCount:{}. "
+ "Published tombstone for orphan service units: serviceUnitTombstoneCnt:{}, "
+ "serviceUnitTombstoneErrorCnt:{}, metrics:{} ",
+ "approximate cleanupErrorCnt:{}, metrics:{} ",
monitorTime,
inactiveBrokers,
inactiveBrokers.size(),
serviceUnitTombstoneCnt,
serviceUnitTombstoneErrorCnt,
totalCleanupErrorCntStart - totalCleanupErrorCnt.get(),
printCleanupMetrics());

}

private String printCleanupMetrics() {
return String.format(
"{totalCleanupCnt:%d, totalBrokerCleanupTombstoneCnt:%d, "
+ "totalServiceUnitCleanupTombstoneCnt:%d, totalServiceUnitCleanupErrorCnt:%d, "
+ "totalServiceUnitCleanupTombstoneCnt:%d, totalCleanupErrorCnt:%d, "
+ "totalCleanupScheduledCnt%d, totalCleanupIgnoredCnt:%d, totalCleanupCancelledCnt:%d, "
+ " activeCleanupJobs:%d}",
totalCleanupCnt,
totalBrokerCleanupTombstoneCnt,
totalServiceUnitCleanupTombstoneCnt,
totalServiceUnitCleanupErrorCnt,
totalCleanupErrorCnt.get(),
totalCleanupScheduledCnt,
totalCleanupIgnoredCnt,
totalCleanupCancelledCnt,
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,89 @@
/*
* 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 static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Assigned;
import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Free;
import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Owned;
import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Released;
import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Splitting;
import com.google.common.annotations.VisibleForTesting;
import org.apache.commons.lang.StringUtils;
import org.apache.pulsar.client.api.Schema;
import org.apache.pulsar.common.topics.TopicCompactionStrategy;

public class ServiceUnitStateCompactionStrategy implements TopicCompactionStrategy<ServiceUnitStateData> {

private final Schema<ServiceUnitStateData> schema;

private boolean checkBrokers = true;

public ServiceUnitStateCompactionStrategy() {
schema = Schema.JSON(ServiceUnitStateData.class);
}

@Override
public Schema<ServiceUnitStateData> getSchema() {
return schema;
}

@VisibleForTesting
public void checkBrokers(boolean check) {
this.checkBrokers = check;
}

@Override
public boolean shouldKeepLeft(ServiceUnitStateData from, ServiceUnitStateData to) {
ServiceUnitState prevState = from == null ? Free : from.state();
ServiceUnitState state = to == null ? Free : to.state();
if (!ServiceUnitState.isValidTransition(prevState, state)) {
return true;
}

if (checkBrokers) {
if (prevState == Free && (state == Assigned || state == Owned)) {
// Free -> Assigned || Owned broker check
return StringUtils.isBlank(to.broker());
} else if (prevState == Owned && state == Assigned) {
// Owned -> Assigned(transfer) broker check
return !StringUtils.equals(from.broker(), to.sourceBroker())
|| StringUtils.isBlank(to.broker())
|| StringUtils.equals(from.broker(), to.broker());
} else if (prevState == Assigned && state == Released) {
// Assigned -> Released(transfer) broker check
return !StringUtils.equals(from.broker(), to.broker())
|| !StringUtils.equals(from.sourceBroker(), to.sourceBroker());
} else if (prevState == Released && state == Owned) {
// Released -> Owned(transfer) broker check
return !StringUtils.equals(from.broker(), to.broker())
|| !StringUtils.equals(from.sourceBroker(), to.sourceBroker());
} else if (prevState == Assigned && state == Owned) {
// Assigned -> Owned broker check
return !StringUtils.equals(from.broker(), to.broker())
|| !StringUtils.equals(from.sourceBroker(), to.sourceBroker());
} else if (prevState == Owned && state == Splitting) {
// Owned -> Splitting broker check
return !StringUtils.equals(from.broker(), to.broker());
}
}

return false;
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -78,6 +78,8 @@
import org.apache.commons.collections4.CollectionUtils;
import org.apache.commons.lang3.StringUtils;
import org.apache.pulsar.broker.PulsarServerException;
import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl;
import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateCompactionStrategy;
import org.apache.pulsar.broker.namespace.NamespaceService;
import org.apache.pulsar.broker.resources.NamespaceResources.PartitionedTopicResources;
import org.apache.pulsar.broker.service.AbstractReplicator;
Expand Down Expand Up @@ -152,6 +154,7 @@
import org.apache.pulsar.common.policies.data.stats.TopicStatsImpl;
import org.apache.pulsar.common.protocol.Commands;
import org.apache.pulsar.common.protocol.schema.SchemaData;
import org.apache.pulsar.common.topics.TopicCompactionStrategy;
import org.apache.pulsar.common.util.Codec;
import org.apache.pulsar.common.util.DateFormatter;
import org.apache.pulsar.common.util.FutureUtil;
Expand Down Expand Up @@ -203,6 +206,11 @@ public class PersistentTopic extends AbstractTopic implements Topic, AddEntryCal
private CompletableFuture<Long> currentCompaction = CompletableFuture.completedFuture(COMPACTION_NEVER_RUN);
private final CompactedTopic compactedTopic;

// TODO: Create compaction strategy from topic policy when exposing strategic compaction to users.
private static Map<String, TopicCompactionStrategy> strategicCompactionMap = Map.of(
ServiceUnitStateChannelImpl.TOPIC,
new ServiceUnitStateCompactionStrategy());

private CompletableFuture<MessageIdImpl> currentOffload = CompletableFuture.completedFuture(
(MessageIdImpl) MessageId.earliest);

Expand Down Expand Up @@ -1571,6 +1579,11 @@ public void checkCompaction() {
}

if (backlogEstimate > compactionThreshold) {
if (log.isDebugEnabled()) {
log.debug(
"topic:{} backlogEstimate:{} is bigger than compactionThreshold:{}. Triggering "
+ "compaction", topic, backlogEstimate, compactionThreshold);
}
try {
triggerCompaction();
} catch (AlreadyRunningException are) {
Expand Down Expand Up @@ -3000,7 +3013,13 @@ public void readEntryFailed(ManagedLedgerException exception, Object ctx) {
public synchronized void triggerCompaction()
throws PulsarServerException, AlreadyRunningException {
if (currentCompaction.isDone()) {
currentCompaction = brokerService.pulsar().getCompactor().compact(topic);

if (strategicCompactionMap.containsKey(topic)) {
currentCompaction = brokerService.pulsar().getStrategicCompactor()
.compact(topic, strategicCompactionMap.get(topic));
} else {
currentCompaction = brokerService.pulsar().getCompactor().compact(topic);
}
} else {
throw new AlreadyRunningException("Compaction already in progress");
}
Expand Down
Loading

0 comments on commit 1cd1aef

Please sign in to comment.