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, // 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 { } else {
intervalToNumShards = CollectionUtils.mapValues( intervalToNumShards = CollectionUtils.mapValues(

View File

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

View File

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

View File

@ -185,7 +185,9 @@ public class OverlordResourceTestClient
HttpMethod.GET, HttpMethod.GET,
StringUtils.format("%s%s", getIndexerURL(), identifier) 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( return jsonMapper.readValue(
response.getContent(), new TypeReference<List<TaskResponseObject>>() response.getContent(), new TypeReference<List<TaskResponseObject>>()
{ {
@ -204,7 +206,9 @@ public class OverlordResourceTestClient
HttpMethod.GET, HttpMethod.GET,
StringUtils.format("%stask/%s", getIndexerURL(), StringUtils.urlEncode(taskId)) 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( return jsonMapper.readValue(
response.getContent(), new TypeReference<TaskPayloadResponse>() response.getContent(), new TypeReference<TaskPayloadResponse>()
{ {

View File

@ -595,7 +595,6 @@ public class FrameProcessorExecutor
sb.append("; cancel=").append(finishedFuture.isCancelled() ? "y" : "n"); sb.append("; cancel=").append(finishedFuture.isCancelled() ? "y" : "n");
sb.append("; done=").append(finishedFuture.isDone() ? "y" : "n"); sb.append("; done=").append(finishedFuture.isDone() ? "y" : "n");
log.debug(StringUtils.encodeForFormat(sb.toString())); 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); final int computedNumParallelTasks = Math.max(computedOptimalParallelism, 1);
LOG.debug( if (LOG.isDebugEnabled()) {
"Computed parallel tasks: [%s]; ForkJoinPool details - sequence parallelism: [%s] " ForkJoinPool pool = getPool();
+ "active threads: [%s] running threads: [%s] queued submissions: [%s] queued tasks: [%s] " LOG.debug(
+ "pool parallelism: [%s] pool size: [%s] steal count: [%s]", "Computed parallel tasks: [%s]; ForkJoinPool details - sequence parallelism: [%s] "
computedNumParallelTasks, + "active threads: [%s] running threads: [%s] queued submissions: [%s] queued tasks: [%s] "
parallelism, + "pool parallelism: [%s] pool size: [%s] steal count: [%s]",
getPool().getActiveThreadCount(), computedNumParallelTasks, parallelism,
runningThreadCount, pool.getActiveThreadCount(), runningThreadCount, submissionCount, pool.getQueuedTaskCount(),
submissionCount, pool.getParallelism(), pool.getPoolSize(), pool.getStealCount()
getPool().getQueuedTaskCount(), );
getPool().getParallelism(), }
getPool().getPoolSize(),
getPool().getStealCount()
);
return computedNumParallelTasks; return computedNumParallelTasks;
} }

View File

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

View File

@ -91,7 +91,9 @@ public class DataServerClient
requestBuilder = requestBuilder.jsonContent(objectMapper, query); 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( ListenableFuture<InputStream> resultStreamFuture = serviceClient.asyncRequest(
requestBuilder, requestBuilder,
new DataServerResponseHandler(query, responseContext, objectMapper) new DataServerResponseHandler(query, responseContext, objectMapper)

View File

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

View File

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