abhishekrb19 commented on code in PR #17353: URL: https://github.com/apache/druid/pull/17353#discussion_r1950111507
########## indexing-service/src/main/java/org/apache/druid/indexing/scheduledbatch/ScheduledBatchStatusTracker.java: ########## @@ -0,0 +1,149 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.scheduledbatch; + +import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.query.http.SqlTaskStatus; +import org.joda.time.DateTime; +import org.joda.time.Duration; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.CopyOnWriteArrayList; +import java.util.concurrent.atomic.AtomicInteger; + +/** + * Tracks task statuses upon submission and completion for scheduled batch supervisors. + * <p> + * This class maintains per-supervisor mappings of submitted tasks and their statuses. + * It also keeps track of total task counts (active, successful, failed) and ensures that + * recently tracked task statuses are retained for a limited duration ({@link #MAX_STATUS_RETAIN_DURATION}). + * </p> + */ +public class ScheduledBatchStatusTracker +{ + private static final Duration MAX_STATUS_RETAIN_DURATION = Duration.standardDays(2); + + /** + * Track supervisor ID -> task IDs. + */ + private final ConcurrentHashMap<String, List<String>> supervisorToTaskIds = new ConcurrentHashMap<>(); + + /** + * Track the task ID -> supervisor ID for reverse lookup to update the total counts. + */ + private final ConcurrentHashMap<String, String> taskIdToSupervisorId = new ConcurrentHashMap<>(); + + /** + * Tracks the recent set of task ID -> task status for all tasks younger than {@link #MAX_STATUS_RETAIN_DURATION}. + */ + private final ConcurrentHashMap<String, BatchSupervisorTaskStatus> recentTaskStatusMap = new ConcurrentHashMap<>(); + + + private final ConcurrentHashMap<String, AtomicInteger> supervisorTotalSubmittedTasks = new ConcurrentHashMap<>(); + private final ConcurrentHashMap<String, AtomicInteger> supervisorTotalSuccessfulTasks = new ConcurrentHashMap<>(); + private final ConcurrentHashMap<String, AtomicInteger> supervisorTotalFailedTasks = new ConcurrentHashMap<>(); Review Comment: These 3 aggregate maps have now been moved inside a small wrapper `SupervisorStatusSummary` (for each supervisor). The `recentTaskStatusMap` remains outside to allow faster lookups in `onTaskCompleted()` so we can return quickly if it's not a task submitted by the batch supervisor. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
