Skip to content

Commit 659074e

Browse files
committed
[improve][broker] PIP-192 added operation counters in ServiceUnitStateChannel
1 parent 31fe347 commit 659074e

File tree

2 files changed

+313
-40
lines changed

2 files changed

+313
-40
lines changed

pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateChannelImpl.java

+148-39
Original file line numberDiff line numberDiff line change
@@ -27,6 +27,9 @@
2727
import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl.ChannelState.Constructed;
2828
import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl.ChannelState.LeaderElectionServiceStarted;
2929
import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl.ChannelState.Started;
30+
import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl.EventType.Assign;
31+
import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl.EventType.Split;
32+
import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl.EventType.Unload;
3033
import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl.MetadataState.Jittery;
3134
import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl.MetadataState.Stable;
3235
import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl.MetadataState.Unstable;
@@ -44,6 +47,8 @@
4447
import java.util.concurrent.TimeUnit;
4548
import java.util.concurrent.TimeoutException;
4649
import java.util.concurrent.atomic.AtomicLong;
50+
import lombok.AllArgsConstructor;
51+
import lombok.Getter;
4752
import lombok.extern.slf4j.Slf4j;
4853
import org.apache.commons.lang3.StringUtils;
4954
import org.apache.commons.lang3.mutable.MutableInt;
@@ -107,6 +112,40 @@ public class ServiceUnitStateChannelImpl implements ServiceUnitStateChannel {
107112
private long totalCleanupCancelledCnt = 0;
108113
private volatile ChannelState channelState;
109114

115+
enum EventType {
116+
Assign,
117+
Split,
118+
Unload
119+
}
120+
121+
@Getter
122+
@AllArgsConstructor
123+
static class Counters {
124+
private AtomicLong total;
125+
private AtomicLong failure;
126+
}
127+
128+
// operation metrics
129+
final Map<ServiceUnitState, AtomicLong> ownerLookUpCounters = Map.of(
130+
Owned, new AtomicLong(),
131+
Assigned, new AtomicLong(),
132+
Released, new AtomicLong(),
133+
Splitting, new AtomicLong(),
134+
Free, new AtomicLong()
135+
);
136+
final Map<EventType, Counters> eventCounters = Map.of(
137+
Assign, new Counters(new AtomicLong(), new AtomicLong()),
138+
Split, new Counters(new AtomicLong(), new AtomicLong()),
139+
Unload, new Counters(new AtomicLong(), new AtomicLong())
140+
);
141+
final Map<ServiceUnitState, Counters> handlerCounters = Map.of(
142+
Owned, new Counters(new AtomicLong(), new AtomicLong()),
143+
Assigned, new Counters(new AtomicLong(), new AtomicLong()),
144+
Released, new Counters(new AtomicLong(), new AtomicLong()),
145+
Splitting, new Counters(new AtomicLong(), new AtomicLong()),
146+
Free, new Counters(new AtomicLong(), new AtomicLong())
147+
);
148+
110149
enum ChannelState {
111150
Closed(0),
112151
Constructed(1),
@@ -279,67 +318,112 @@ private boolean isChannelOwner() {
279318
public CompletableFuture<String> getOwnerAsync(String serviceUnit) {
280319
validateChannelState(Started, true);
281320
ServiceUnitStateData data = tableview.get(serviceUnit);
282-
if (data == null) {
283-
return CompletableFuture.completedFuture(null);
284-
}
285-
switch (data.state()) {
321+
ServiceUnitState state = data == null ? Free : data.state();
322+
ownerLookUpCounters.get(state).incrementAndGet();
323+
switch (state) {
286324
case Owned, Splitting -> {
287325
return CompletableFuture.completedFuture(data.broker());
288326
}
289327
case Assigned, Released -> {
290328
return deferGetOwnerRequest(serviceUnit);
291329
}
330+
case Free -> {
331+
return CompletableFuture.completedFuture(null);
332+
}
292333
default -> {
293-
return null;
334+
throw new IllegalStateException("Invalid service unit state:" + data.state());
294335
}
295336
}
296337
}
297338

298339
public CompletableFuture<String> publishAssignEventAsync(String serviceUnit, String broker) {
299-
CompletableFuture<String> getOwnerRequest = deferGetOwnerRequest(serviceUnit);
300-
pubAsync(serviceUnit, new ServiceUnitStateData(Assigned, broker))
301-
.whenComplete((__, ex) -> {
302-
if (ex != null) {
303-
getOwnerRequests.remove(serviceUnit, getOwnerRequest);
304-
if (!getOwnerRequest.isCompletedExceptionally()) {
305-
getOwnerRequest.completeExceptionally(ex);
340+
EventType eventType = Assign;
341+
eventCounters.get(eventType).getTotal().incrementAndGet();
342+
try {
343+
CompletableFuture<String> getOwnerRequest = deferGetOwnerRequest(serviceUnit);
344+
pubAsync(serviceUnit, new ServiceUnitStateData(Assigned, broker))
345+
.whenComplete((__, ex) -> {
346+
if (ex != null) {
347+
getOwnerRequests.remove(serviceUnit, getOwnerRequest);
348+
if (!getOwnerRequest.isCompletedExceptionally()) {
349+
getOwnerRequest.completeExceptionally(ex);
350+
}
351+
eventCounters.get(eventType).getFailure().incrementAndGet();
306352
}
307-
}
308-
});
309-
310-
return getOwnerRequest;
353+
});
354+
return getOwnerRequest;
355+
} catch (Throwable e) {
356+
log.error("Failed to publish assign event. serviceUnit:{}, broker:{}, assignPublishFailureCount:{}",
357+
serviceUnit, broker, eventCounters.get(eventType).getFailure().incrementAndGet(), e);
358+
throw e;
359+
}
311360
}
312361

313362
public CompletableFuture<Void> publishUnloadEventAsync(Unload unload) {
314-
String serviceUnit = unload.serviceUnit();
315-
if (isTransferCommand(unload)) {
316-
ServiceUnitStateData next = new ServiceUnitStateData(Assigned,
317-
unload.destBroker().get(), unload.sourceBroker());
318-
return pubAsync(serviceUnit, next).thenApply(__ -> null);
363+
EventType eventType = Unload;
364+
eventCounters.get(eventType).getTotal().incrementAndGet();
365+
try {
366+
String serviceUnit = unload.serviceUnit();
367+
CompletableFuture<MessageId> future;
368+
if (isTransferCommand(unload)) {
369+
ServiceUnitStateData next = new ServiceUnitStateData(Assigned,
370+
unload.destBroker().get(), unload.sourceBroker());
371+
future = pubAsync(serviceUnit, next);
372+
} else {
373+
future = tombstoneAsync(serviceUnit);
374+
}
375+
376+
return future.whenComplete((__, ex) -> {
377+
if (ex != null) {
378+
eventCounters.get(eventType).getFailure().incrementAndGet();
379+
}
380+
}).thenApply(__ -> null);
381+
} catch (Throwable e) {
382+
log.error("Failed to publish unload event. unload:{}. unloadPublishFailureCount:{}",
383+
unload, eventCounters.get(eventType).getFailure().incrementAndGet(), e);
384+
throw e;
319385
}
320-
return tombstoneAsync(serviceUnit).thenApply(__ -> null);
321386
}
322387

323388
public CompletableFuture<Void> publishSplitEventAsync(Split split) {
324-
String serviceUnit = split.serviceUnit();
325-
ServiceUnitStateData next = new ServiceUnitStateData(Splitting, split.sourceBroker());
326-
return pubAsync(serviceUnit, next).thenApply(__ -> null);
389+
EventType eventType = Split;
390+
eventCounters.get(eventType).getTotal().incrementAndGet();
391+
try {
392+
String serviceUnit = split.serviceUnit();
393+
ServiceUnitStateData next = new ServiceUnitStateData(Splitting, split.sourceBroker());
394+
return pubAsync(serviceUnit, next).whenComplete((__, ex) -> {
395+
if (ex != null) {
396+
eventCounters.get(eventType).getFailure().incrementAndGet();
397+
}
398+
}).thenApply(__ -> null);
399+
} catch (Throwable e) {
400+
log.error("Failed to publish split event. split:{}, splitPublishFailureCount:{}",
401+
split, eventCounters.get(eventType).getFailure().incrementAndGet(), e);
402+
throw e;
403+
}
327404
}
328405

329406
private void handle(String serviceUnit, ServiceUnitStateData data) {
407+
long totalHandledRequests = getHandlerTotalCounter(data).incrementAndGet();
330408
if (log.isDebugEnabled()) {
331-
log.info("{} received a handle request for serviceUnit:{}, data:{}",
332-
lookupServiceAddress, serviceUnit, data);
409+
log.info("{} received a handle request for serviceUnit:{}, data:{}. totalHandledRequests:{}",
410+
lookupServiceAddress, serviceUnit, data, totalHandledRequests);
333411
}
334412

335413
ServiceUnitState state = data == null ? Free : data.state();
336-
switch (state) {
337-
case Owned -> handleOwnEvent(serviceUnit, data);
338-
case Assigned -> handleAssignEvent(serviceUnit, data);
339-
case Released -> handleReleaseEvent(serviceUnit, data);
340-
case Splitting -> handleSplitEvent(serviceUnit, data);
341-
case Free -> handleFreeEvent(serviceUnit);
342-
default -> throw new IllegalStateException("Failed to handle channel data:" + data);
414+
try {
415+
switch (state) {
416+
case Owned -> handleOwnEvent(serviceUnit, data);
417+
case Assigned -> handleAssignEvent(serviceUnit, data);
418+
case Released -> handleReleaseEvent(serviceUnit, data);
419+
case Splitting -> handleSplitEvent(serviceUnit, data);
420+
case Free -> handleFreeEvent(serviceUnit);
421+
default -> throw new IllegalStateException("Failed to handle channel data:" + data);
422+
}
423+
} catch (Throwable e){
424+
getHandlerFailureCounter(data).incrementAndGet();
425+
log.error("Failed to handle the event. serviceUnit:{}, data:{}", serviceUnit, data, e);
426+
throw e;
343427
}
344428
}
345429

@@ -359,19 +443,46 @@ private static String getLogEventTag(ServiceUnitStateData data) {
359443
isTransferCommand(data) ? "Transfer:" + data.state() : data.state().toString();
360444
}
361445

446+
private AtomicLong getHandlerTotalCounter(ServiceUnitStateData data) {
447+
return getHandlerCounter(data, true);
448+
}
449+
450+
private AtomicLong getHandlerFailureCounter(ServiceUnitStateData data) {
451+
return getHandlerCounter(data, false);
452+
}
453+
454+
private AtomicLong getHandlerCounter(ServiceUnitStateData data, boolean total) {
455+
var state = data.state() == null ? Free : data.state();
456+
var counter = total
457+
? handlerCounters.get(state).getTotal() : handlerCounters.get(state).getFailure();
458+
if (counter == null) {
459+
throw new IllegalStateException("Unknown state:" + state);
460+
}
461+
return counter;
462+
}
463+
362464
private void log(Throwable e, String serviceUnit, ServiceUnitStateData data, ServiceUnitStateData next) {
363465
if (e == null) {
364466
if (log.isDebugEnabled() || isTransferCommand(data)) {
365-
log.info("{} handled {} event for serviceUnit:{}, cur:{}, next:{}",
467+
long handlerTotalCount = getHandlerTotalCounter(data).get();
468+
long handlerFailureCount = getHandlerFailureCounter(data).get();
469+
log.info("{} handled {} event for serviceUnit:{}, cur:{}, next:{}, "
470+
+ "totalHandledRequests{}, totalFailedRequests:{}",
366471
lookupServiceAddress, getLogEventTag(data), serviceUnit,
367472
data == null ? "" : data,
368-
next == null ? "" : next);
473+
next == null ? "" : next,
474+
handlerTotalCount, handlerFailureCount
475+
);
369476
}
370477
} else {
371-
log.error("{} failed to handle {} event for serviceUnit:{}, cur:{}, next:{}",
478+
long handlerTotalCount = getHandlerTotalCounter(data).get();
479+
long handlerFailureCount = getHandlerFailureCounter(data).incrementAndGet();
480+
log.error("{} failed to handle {} event for serviceUnit:{}, cur:{}, next:{}, "
481+
+ "totalHandledRequests{}, totalFailedRequests:{}",
372482
lookupServiceAddress, getLogEventTag(data), serviceUnit,
373483
data == null ? "" : data,
374484
next == null ? "" : next,
485+
handlerTotalCount, handlerFailureCount,
375486
e);
376487
}
377488
}
@@ -384,7 +495,6 @@ private void handleOwnEvent(String serviceUnit, ServiceUnitStateData data) {
384495
if (isTargetBroker(data.broker())) {
385496
log(null, serviceUnit, data, null);
386497
}
387-
388498
}
389499

390500
private void handleAssignEvent(String serviceUnit, ServiceUnitStateData data) {
@@ -398,7 +508,6 @@ private void handleAssignEvent(String serviceUnit, ServiceUnitStateData data) {
398508
}
399509

400510
private void handleReleaseEvent(String serviceUnit, ServiceUnitStateData data) {
401-
402511
if (isTargetBroker(data.sourceBroker())) {
403512
ServiceUnitStateData next = new ServiceUnitStateData(Owned, data.broker(), data.sourceBroker());
404513
// TODO: when close, pass message to clients to connect to the new broker

0 commit comments

Comments
 (0)