Skip to content

Commit

Permalink
Optimize the debug log that affects performance, and unify the style (#…
Browse files Browse the repository at this point in the history
  • Loading branch information
liudezhi2098 authored Dec 27, 2021
1 parent 47c7cd9 commit fb4e2c8
Show file tree
Hide file tree
Showing 5 changed files with 29 additions and 13 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -182,8 +182,10 @@ public synchronized CompletableFuture<Void> disconnect(boolean failIfHasBacklog)
if (failIfHasBacklog && getNumberOfEntriesInBacklog() > 0) {
CompletableFuture<Void> disconnectFuture = new CompletableFuture<>();
disconnectFuture.completeExceptionally(new TopicBusyException("Cannot close a replicator with backlog"));
log.debug("[{}][{} -> {}] Replicator disconnect failed since topic has backlog", topicName, localCluster,
remoteCluster);
if (log.isDebugEnabled()) {
log.debug("[{}][{} -> {}] Replicator disconnect failed since topic has backlog", topicName, localCluster
, remoteCluster);
}
return disconnectFuture;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -105,7 +105,9 @@ public void add(T event, long ts, Record<?> record) {
public void add(Event<T> windowEvent) {
// watermark events are not added to the queue.
if (windowEvent.isWatermark()) {
log.debug(String.format("Got watermark event with ts %d", windowEvent.getTimestamp()));
if (log.isDebugEnabled()) {
log.debug("Got watermark event with ts {}", windowEvent.getTimestamp());
}
} else {
queue.add(windowEvent);
}
Expand Down Expand Up @@ -145,8 +147,9 @@ public boolean onTrigger() {
prevWindowEvents.clear();
if (!events.isEmpty()) {
prevWindowEvents.addAll(windowEvents);
log.debug(String.format("invoking windowLifecycleListener onActivation, [%d] events in "
+ "window.", events.size()));
if (log.isDebugEnabled()) {
log.debug("invoking windowLifecycleListener onActivation, [{}] events in window.", events.size());
}
windowLifecycleListener.onActivation(events, newEvents, expired,
evictionPolicy.getContext().getReferenceTime());
} else {
Expand Down Expand Up @@ -216,7 +219,9 @@ private List<Event<T>> scanEvents(boolean fullScan) {
lock.unlock();
}
eventsSinceLastExpiry.set(0);
log.debug(String.format("[%d] events expired from window.", eventsToExpire.size()));
if (log.isDebugEnabled()) {
log.debug("[{}] events expired from window.", eventsToExpire.size());
}
if (!eventsToExpire.isEmpty()) {
log.debug("invoking windowLifecycleListener.onExpiry");
windowLifecycleListener.onExpiry(eventsToExpire);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -80,7 +80,9 @@ public void shutdown() {
private void handleWaterMarkEvent(Event<T> event) {
long watermarkTs = event.getTimestamp();
long windowEndTs = nextWindowEndTs;
log.debug(String.format("Window end ts %d Watermark ts %d", windowEndTs, watermarkTs));
if (log.isDebugEnabled()) {
log.debug("Window end ts {} Watermark ts {}", windowEndTs, watermarkTs);
}
while (windowEndTs <= watermarkTs) {
long currentCount = windowManager.getEventCount(windowEndTs);
evictionPolicy.setContext(new DefaultEvictionContext(windowEndTs, currentCount));
Expand All @@ -93,10 +95,14 @@ private void handleWaterMarkEvent(Event<T> event) {
* window intervals based on event ts.
*/
long ts = getNextAlignedWindowTs(windowEndTs, watermarkTs);
log.debug(String.format("Next aligned window end ts %d", ts));
if (log.isDebugEnabled()) {
log.debug("Next aligned window end ts {}", ts);
}
if (ts == Long.MAX_VALUE) {
log.debug(String.format("No events to process between %d and watermark ts %d",
windowEndTs, watermarkTs));
if (log.isDebugEnabled()) {
log.debug("No events to process between {} and watermark ts {}",
windowEndTs, watermarkTs);
}
break;
}
windowEndTs = ts;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -413,8 +413,9 @@ synchronized boolean processDeregister(String tenant, String namespace,
String functionName, long version) throws IllegalArgumentException {

boolean needsScheduling = false;

log.debug("Process deregister request: {}/{}/{}/{}", tenant, namespace, functionName, version);
if (log.isDebugEnabled()) {
log.debug("Process deregister request: {}/{}/{}/{}", tenant, namespace, functionName, version);
}

// Check if we still have this function. Maybe already deleted by someone else
if (this.containsFunctionMetaData(tenant, namespace, functionName)) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -437,7 +437,9 @@ public boolean hasFinished() {

@Override
public void readEntriesFailed(ManagedLedgerException exception, Object ctx) {
log.debug(exception, "Failed to read entries from topic %s", topicName.toString());
if (log.isDebugEnabled()) {
log.debug(exception, "Failed to read entries from topic %s", topicName.toString());
}
outstandingReadsRequests.incrementAndGet();

//set read latency stats for failed
Expand Down

0 comments on commit fb4e2c8

Please sign in to comment.