Skip to content

Commit

Permalink
Clean up allocation and supervisor logs for easier debugging (#16535)
Browse files Browse the repository at this point in the history
Changes:
- Use string taskGroup consistently to easily search for a task group
- Clean up other logs
- No change in any logic
  • Loading branch information
kfaraz authored Jun 3, 2024
1 parent d091686 commit 1974a38
Show file tree
Hide file tree
Showing 3 changed files with 51 additions and 62 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -268,7 +268,7 @@ private void processBatchesDue()
catch (Throwable t) {
currentBatch.failPendingRequests(t);
processed = true;
log.error(t, "Error while processing batch [%s]", currentBatch.key);
log.error(t, "Error while processing batch[%s].", currentBatch.key);
}

// Requeue if not fully processed yet
Expand Down Expand Up @@ -619,7 +619,7 @@ void failPendingRequests(String reason)
void failPendingRequests(Throwable cause)
{
if (!requestToFuture.isEmpty()) {
log.warn("Failing [%d] requests in batch [%s], reason [%s].", size(), cause.getMessage(), key);
log.warn("Failing [%d] requests in batch[%s], reason[%s].", size(), key, cause.getMessage());
requestToFuture.values().forEach(future -> future.completeExceptionally(cause));
requestToFuture.keySet().forEach(
request -> emitTaskMetric("task/action/failed/count", 1L, request)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -657,7 +657,7 @@ private TaskLockPosse createOrFindLockPosse(LockRequest request, Task task, bool
}

} else {
log.info("Task[%s] already present in TaskLock[%s]", task.getId(), posseToUse.getTaskLock().getGroupId());
log.debug("Task[%s] already present in TaskLock[%s].", task.getId(), posseToUse.getTaskLock().getGroupId());
}
return posseToUse;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -83,6 +83,7 @@
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.Pair;
import org.apache.druid.java.util.common.RetryUtils;
import org.apache.druid.java.util.common.Stopwatch;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.concurrent.Execs;
import org.apache.druid.java.util.emitter.EmittingLogger;
Expand All @@ -96,13 +97,12 @@
import org.apache.druid.segment.incremental.RowIngestionMetersFactory;
import org.apache.druid.segment.indexing.DataSchema;
import org.joda.time.DateTime;
import org.joda.time.Duration;

import javax.annotation.Nonnull;
import javax.annotation.Nullable;
import javax.validation.constraints.NotNull;
import java.io.IOException;
import java.time.Duration;
import java.time.Instant;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
Expand Down Expand Up @@ -1203,19 +1203,17 @@ public void tryInit()
}

try {
Instant handleNoticeStartTime = Instant.now();
final Stopwatch noticeHandleTime = Stopwatch.createStarted();
notice.handle();
Instant handleNoticeEndTime = Instant.now();
Duration timeElapsed = Duration.between(handleNoticeStartTime, handleNoticeEndTime);
String noticeType = notice.getType();
emitNoticeProcessTime(noticeType, noticeHandleTime.millisElapsed());
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
"Handled notice[%s] from notices queue in [%d] ms, "
+ "current notices queue size [%d] for datasource[%s].",
noticeType, noticeHandleTime.millisElapsed(), getNoticesQueueSize(), dataSource
);
}
emitNoticeProcessTime(noticeType, timeElapsed.toMillis());
}
catch (Throwable e) {
stateManager.recordThrowableEvent(e);
Expand Down Expand Up @@ -2837,10 +2835,7 @@ private boolean updatePartitionDataFromStream()
earlyStopTime = DateTimes.nowUtc().plus(tuningConfig.getRepartitionTransitionDuration());
log.info(
"Previous partition set [%s] has changed to [%s] - requesting that tasks stop after [%s] at [%s]",
previousPartitionIds,
partitionIds,
tuningConfig.getRepartitionTransitionDuration(),
earlyStopTime
previousPartitionIds, partitionIds, tuningConfig.getRepartitionTransitionDuration(), earlyStopTime
);
break;
}
Expand Down Expand Up @@ -3161,57 +3156,52 @@ private void checkTaskDuration() throws ExecutionException, InterruptedException
final List<ListenableFuture<Map<PartitionIdType, SequenceOffsetType>>> futures = new ArrayList<>();
final List<Integer> futureGroupIds = new ArrayList<>();

boolean stopTasksEarly;
if (earlyStopTime != null && (earlyStopTime.isBeforeNow() || earlyStopTime.isEqualNow())) {
log.info("Early stop requested - signalling tasks to complete");

final boolean stopTasksEarly;
if (earlyStopTime != null && !earlyStopTime.isAfterNow()) {
log.info("Early stop requested, signalling tasks to complete.");
earlyStopTime = null;
stopTasksEarly = true;
} else {
stopTasksEarly = false;
}

AtomicInteger stoppedTasks = new AtomicInteger();
final AtomicInteger numStoppedTasks = new AtomicInteger();
// Sort task groups by start time to prioritize early termination of earlier groups, then iterate for processing
activelyReadingTaskGroups
.entrySet().stream().sorted(
activelyReadingTaskGroups.entrySet().stream().sorted(
Comparator.comparingLong(
(Entry<Integer, TaskGroup> entry) ->
computeEarliestTaskStartTime(entry.getValue())
.getMillis()))
taskGroupEntry -> computeEarliestTaskStartTime(taskGroupEntry.getValue()).getMillis()
)
)
.forEach(entry -> {
Integer groupId = entry.getKey();
TaskGroup group = entry.getValue();

if (stopTasksEarly) {
final DateTime earliestTaskStart = computeEarliestTaskStartTime(group);
final Duration runDuration = Duration.millis(DateTimes.nowUtc().getMillis() - earliestTaskStart.getMillis());
if (stopTasksEarly || group.getHandoffEarly()) {
// If handoffEarly has been set, stop tasks irrespective of stopTaskCount
log.info(
"Stopping task group [%d] early. It has run for [%s]",
groupId,
ioConfig.getTaskDuration()
"Stopping taskGroup[%d] early after running for duration[%s].",
groupId, runDuration
);
futureGroupIds.add(groupId);
futures.add(checkpointTaskGroup(group, true));
} else {
DateTime earliestTaskStart = computeEarliestTaskStartTime(group);

if (earliestTaskStart.plus(ioConfig.getTaskDuration()).isBeforeNow() || group.getHandoffEarly()) {
// if this task has run longer than the configured duration
// as long as the pending task groups are less than the configured stop task count.
// If shutdownEarly has been set, ignore stopTaskCount since this is a manual operator action.
if (pendingCompletionTaskGroups.values()
.stream()
.mapToInt(CopyOnWriteArrayList::size)
.sum() + stoppedTasks.get()
< ioConfig.getMaxAllowedStops() || group.getHandoffEarly()) {
log.info(
"Task group [%d] has run for [%s]. Stopping.",
groupId,
ioConfig.getTaskDuration()
);
futureGroupIds.add(groupId);
futures.add(checkpointTaskGroup(group, true));
stoppedTasks.getAndIncrement();
}
if (group.getHandoffEarly()) {
numStoppedTasks.getAndIncrement();
}
} else if (earliestTaskStart.plus(ioConfig.getTaskDuration()).isBeforeNow()) {
// Stop this task group if it has run longer than the configured duration
// and the pending task groups are less than the configured stop task count.
int numPendingCompletionTaskGroups = pendingCompletionTaskGroups.values().stream()
.mapToInt(List::size).sum();
if (numPendingCompletionTaskGroups + numStoppedTasks.get() < ioConfig.getMaxAllowedStops()) {
log.info(
"Stopping taskGroup[%d] as it has already run for duration[%s], configured task duration[%s].",
groupId, runDuration, ioConfig.getTaskDuration()
);
futureGroupIds.add(groupId);
futures.add(checkpointTaskGroup(group, true));
numStoppedTasks.getAndIncrement();
}
}
});
Expand Down Expand Up @@ -3384,7 +3374,7 @@ public Map<PartitionIdType, SequenceOffsetType> apply(List<Either<Throwable, Map

if (endOffsets.equals(taskGroup.checkpointSequences.lastEntry().getValue())) {
log.warn(
"Checkpoint [%s] is same as the start sequences [%s] of latest sequence for the task group [%d]",
"Checkpoint[%s] is same as the start sequences[%s] of latest sequence for the taskGroup[%d].",
endOffsets,
taskGroup.checkpointSequences.lastEntry().getValue(),
taskGroup.groupId
Expand Down Expand Up @@ -3579,7 +3569,7 @@ private void checkCurrentTaskState() throws ExecutionException, InterruptedExcep
// 2) Remove any tasks that have failed from the list
// 3) If any task completed successfully, stop all the tasks in this group and move to the next group

log.debug("Task group [%d] pre-pruning: %s", groupId, taskGroup.taskIds());
log.debug("taskGroup[%d] pre-pruning: %s.", groupId, taskGroup.taskIds());

Iterator<Entry<String, TaskData>> iTasks = taskGroup.tasks.entrySet().iterator();
while (iTasks.hasNext()) {
Expand All @@ -3589,7 +3579,7 @@ private void checkCurrentTaskState() throws ExecutionException, InterruptedExcep

// stop and remove bad tasks from the task group
if (!isTaskCurrent(groupId, taskId, activeTaskMap)) {
log.info("Stopping task [%s] which does not match the expected sequence range and ingestion spec", taskId);
log.info("Stopping task[%s] as it does not match the expected sequence range and ingestion spec.", taskId);
futures.add(stopTask(taskId, false));
iTasks.remove();
continue;
Expand All @@ -3613,7 +3603,7 @@ private void checkCurrentTaskState() throws ExecutionException, InterruptedExcep
break;
}
}
log.debug("Task group [%d] post-pruning: %s", groupId, taskGroup.taskIds());
log.debug("After pruning, taskGroup[%d] has tasks[%s].", groupId, taskGroup.taskIds());
}

// Ignore return value; just await.
Expand All @@ -3627,10 +3617,9 @@ private void checkIfStreamInactiveAndTurnSupervisorIdle()
}

Map<PartitionIdType, SequenceOffsetType> latestSequencesFromStream = getLatestSequencesFromStream();
long nowTime = Instant.now().toEpochMilli();
boolean idle;
long idleTime;

final long nowTime = DateTimes.nowUtc().getMillis();
final boolean idle;
final long idleTime;
if (lastActiveTimeMillis > 0
&& previousSequencesFromStream.equals(latestSequencesFromStream)
&& computeTotalLag() == 0) {
Expand Down Expand Up @@ -3684,7 +3673,7 @@ private void createNewTasks() throws JsonProcessingException
// check that there is a current task group for each group of partitions in [partitionGroups]
for (Integer groupId : partitionGroups.keySet()) {
if (!activelyReadingTaskGroups.containsKey(groupId)) {
log.info("Creating new task group [%d] for partitions %s", groupId, partitionGroups.get(groupId));
log.info("Creating new taskGroup[%d] for partitions[%s].", groupId, partitionGroups.get(groupId));
Optional<DateTime> minimumMessageTime;
if (ioConfig.getLateMessageRejectionStartDateTime().isPresent()) {
minimumMessageTime = Optional.of(ioConfig.getLateMessageRejectionStartDateTime().get());
Expand Down Expand Up @@ -3771,13 +3760,13 @@ private void createNewTasks() throws JsonProcessingException
if (taskGroup.startingSequences == null ||
taskGroup.startingSequences.size() == 0 ||
taskGroup.startingSequences.values().stream().allMatch(x -> x == null || isEndOfShard(x))) {
log.debug("Nothing to read in any partition for taskGroup [%d], skipping task creation", groupId);
log.debug("Nothing to read in any partition for taskGroup[%d], skipping task creation.", groupId);
continue;
}

if (ioConfig.getReplicas() > taskGroup.tasks.size()) {
log.info(
"Number of tasks [%d] does not match configured numReplicas [%d] in task group [%d], creating more tasks",
"Number of tasks[%d] does not match configured numReplicas[%d] in taskGroup[%d], creating more tasks.",
taskGroup.tasks.size(), ioConfig.getReplicas(), groupId
);
createTasksForGroup(groupId, ioConfig.getReplicas() - taskGroup.tasks.size());
Expand Down

0 comments on commit 1974a38

Please sign in to comment.