Add pre-check for heavy debug logs (#15706)

Co-authored-by: Kashif Faraz <kashif.faraz@gmail.com>
Co-authored-by: Benedict Jin <asdf2014@apache.org>
This commit is contained in:
Sensor 2024-02-29 15:28:14 +08:00 committed by GitHub
parent 7c42e87db9
commit e0bce0ef90
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
10 changed files with 41 additions and 43 deletions

View File

@ -752,7 +752,7 @@ public class ParallelIndexSupervisorTask extends AbstractBatchIndexTask implemen
);
// This is for potential debugging in case we suspect bad estimation of cardinalities etc,
LOG.debug("intervalToNumShards: %s", intervalToNumShards.toString());
LOG.debug("intervalToNumShards: %s", intervalToNumShards);
} else {
intervalToNumShards = CollectionUtils.mapValues(

View File

@ -1155,14 +1155,13 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
Instant handleNoticeEndTime = Instant.now();
Duration timeElapsed = Duration.between(handleNoticeStartTime, handleNoticeEndTime);
String noticeType = notice.getType();
log.debug(
"Handled notice [%s] from notices queue in [%d] ms, "
+ "current notices queue size [%d] for datasource [%s]",
noticeType,
timeElapsed.toMillis(),
getNoticesQueueSize(),
dataSource
);
if (log.isDebugEnabled()) {
log.debug(
"Handled notice [%s] from notices queue in [%d] ms, "
+ "current notices queue size [%d] for datasource [%s]",
noticeType, timeElapsed.toMillis(), getNoticesQueueSize(), dataSource
);
}
emitNoticeProcessTime(noticeType, timeElapsed.toMillis());
}
catch (Throwable e) {

View File

@ -149,7 +149,7 @@ public class LagBasedAutoScaler implements SupervisorTaskAutoScaler
long totalLags = lagStats.getTotalLag();
lagMetricsQueue.offer(totalLags > 0 ? totalLags : 0L);
}
log.debug("Current lags [%s] for dataSource [%s].", new ArrayList<>(lagMetricsQueue), dataSource);
log.debug("Current lags for dataSource[%s] are [%s].", dataSource, lagMetricsQueue);
} else {
log.warn("[%s] supervisor is suspended, skipping lag collection", dataSource);
}

View File

@ -185,7 +185,9 @@ public class OverlordResourceTestClient
HttpMethod.GET,
StringUtils.format("%s%s", getIndexerURL(), identifier)
);
LOG.debug("Tasks %s response %s", identifier, response.getContent());
if (LOG.isDebugEnabled()) {
LOG.debug("Tasks %s response %s", identifier, response.getContent());
}
return jsonMapper.readValue(
response.getContent(), new TypeReference<List<TaskResponseObject>>()
{
@ -204,7 +206,9 @@ public class OverlordResourceTestClient
HttpMethod.GET,
StringUtils.format("%stask/%s", getIndexerURL(), StringUtils.urlEncode(taskId))
);
LOG.debug("Task %s response %s", taskId, response.getContent());
if (LOG.isDebugEnabled()) {
LOG.debug("Task %s response %s", taskId, response.getContent());
}
return jsonMapper.readValue(
response.getContent(), new TypeReference<TaskPayloadResponse>()
{

View File

@ -595,7 +595,6 @@ public class FrameProcessorExecutor
sb.append("; cancel=").append(finishedFuture.isCancelled() ? "y" : "n");
sb.append("; done=").append(finishedFuture.isDone() ? "y" : "n");
log.debug(StringUtils.encodeForFormat(sb.toString()));
}
}

View File

@ -475,20 +475,17 @@ public class ParallelMergeCombiningSequence<T> extends YieldingSequenceBase<T>
final int computedNumParallelTasks = Math.max(computedOptimalParallelism, 1);
LOG.debug(
"Computed parallel tasks: [%s]; ForkJoinPool details - sequence parallelism: [%s] "
+ "active threads: [%s] running threads: [%s] queued submissions: [%s] queued tasks: [%s] "
+ "pool parallelism: [%s] pool size: [%s] steal count: [%s]",
computedNumParallelTasks,
parallelism,
getPool().getActiveThreadCount(),
runningThreadCount,
submissionCount,
getPool().getQueuedTaskCount(),
getPool().getParallelism(),
getPool().getPoolSize(),
getPool().getStealCount()
);
if (LOG.isDebugEnabled()) {
ForkJoinPool pool = getPool();
LOG.debug(
"Computed parallel tasks: [%s]; ForkJoinPool details - sequence parallelism: [%s] "
+ "active threads: [%s] running threads: [%s] queued submissions: [%s] queued tasks: [%s] "
+ "pool parallelism: [%s] pool size: [%s] steal count: [%s]",
computedNumParallelTasks, parallelism,
pool.getActiveThreadCount(), runningThreadCount, submissionCount, pool.getQueuedTaskCount(),
pool.getParallelism(), pool.getPoolSize(), pool.getStealCount()
);
}
return computedNumParallelTasks;
}

View File

@ -490,11 +490,9 @@ public class FileSmoosher implements Closeable
public void close() throws IOException
{
closer.close();
FileSmoosher.LOG.debug(
"Created smoosh file [%s] of size [%s] bytes.",
outFile.getAbsolutePath(),
outFile.length()
);
if (LOG.isDebugEnabled()) {
LOG.debug("Created smoosh file [%s] of size [%s] bytes.", outFile.getAbsolutePath(), outFile.length());
}
}
}
}

View File

@ -91,7 +91,9 @@ public class DataServerClient
requestBuilder = requestBuilder.jsonContent(objectMapper, query);
}
log.debug("Sending request to servers for query[%s], request[%s]", query.getId(), requestBuilder.toString());
if (log.isDebugEnabled()) {
log.debug("Sending request to servers for query[%s], request[%s]", query.getId(), requestBuilder);
}
ListenableFuture<InputStream> resultStreamFuture = serviceClient.asyncRequest(
requestBuilder,
new DataServerResponseHandler(query, responseContext, objectMapper)

View File

@ -538,10 +538,9 @@ public class DruidCoordinator
if (getCompactSegmentsDutyFromCustomGroups().isEmpty()) {
duties.add(compactSegments);
}
log.debug(
"Initialized indexing service duties [%s].",
duties.stream().map(duty -> duty.getClass().getName()).collect(Collectors.toList())
);
if (log.isDebugEnabled()) {
log.debug("Initialized indexing service duties [%s].", duties.stream().map(duty -> duty.getClass().getName()).collect(Collectors.toList()));
}
return ImmutableList.copyOf(duties);
}

View File

@ -230,12 +230,12 @@ public class CuratorLoadQueuePeon implements LoadQueuePeon
final String path = ZKPaths.makePath(basePath, segmentHolder.getSegmentIdentifier());
final byte[] payload = jsonMapper.writeValueAsBytes(segmentHolder.getChangeRequest());
curator.create().withMode(CreateMode.EPHEMERAL).forPath(path, payload);
log.debug(
"ZKNode created for server to [%s] %s [%s]",
basePath,
segmentHolder.getAction(),
segmentHolder.getSegmentIdentifier()
);
if (log.isDebugEnabled()) {
log.debug(
"ZKNode created for server to [%s] %s [%s]",
basePath, segmentHolder.getAction(), segmentHolder.getSegmentIdentifier()
);
}
final ScheduledFuture<?> nodeDeletedCheck = scheduleNodeDeletedCheck(path);
final Stat stat = curator.checkExists().usingWatcher(
(CuratorWatcher) watchedEvent -> {