Copilot commented on code in PR #16299:
URL: https://github.com/apache/pinot/pull/16299#discussion_r2191665257


##########
pinot-core/src/main/java/org/apache/pinot/core/accounting/PerQueryCPUMemAccountantFactory.java:
##########
@@ -452,49 +482,33 @@ public Map<String, AggregatedStats> aggregate(boolean 
isTriggered) {
           String queryId = currentTaskStatus.getQueryId();
           // update inactive queries for cleanInactive()
           _inactiveQuery.remove(queryId);
-          // if triggered, accumulate active query task stats
-          if (isTriggered) {
-            Thread anchorThread = currentTaskStatus.getAnchorThread();
-            boolean isAnchorThread = currentTaskStatus.isAnchorThread();
-            ret.compute(queryId, (k, v) -> v == null
-                ? new AggregatedStats(currentCPUSample, currentMemSample, 
anchorThread,
-                isAnchorThread, threadEntry._errorStatus, queryId)
-                : v.merge(currentCPUSample, currentMemSample, isAnchorThread, 
threadEntry._errorStatus));
+          // If query is in cancelling set, retain it.
+          if (_cancelSentQueries.contains(queryId)) {
+            cancellingQueries.add(queryId);
           }
         }
 
+        Thread thread = entry.getKey();
         if (!thread.isAlive()) {
           _threadEntriesMap.remove(thread);

Review Comment:
   Removing entries from `_threadEntriesMap` inside a for-each over its own 
`entrySet()` can trigger a ConcurrentModificationException. Use an iterator and 
call `iterator.remove()` instead to safely remove dead threads.
   ```suggestion
             iterator.remove();
   ```



##########
pinot-core/src/test/java/org/apache/pinot/core/accounting/TestResourceAccountant.java:
##########
@@ -0,0 +1,94 @@
+/**
+ * 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.pinot.core.accounting;
+
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.CountDownLatch;
+import java.util.stream.Collectors;
+import org.apache.pinot.spi.accounting.ThreadExecutionContext;
+import org.apache.pinot.spi.config.instance.InstanceType;
+import org.apache.pinot.spi.env.PinotConfiguration;
+import org.apache.pinot.spi.utils.CommonConstants;
+
+
+class TestResourceAccountant extends 
PerQueryCPUMemAccountantFactory.PerQueryCPUMemResourceUsageAccountant {
+  TestResourceAccountant(Map<Thread, 
CPUMemThreadLevelAccountingObjects.ThreadEntry> threadEntries) {
+    super(new PinotConfiguration(), false, true, true, new HashSet<>(), 
"test", InstanceType.SERVER);
+    _threadEntriesMap.putAll(threadEntries);
+  }
+
+  static void getQueryThreadEntries(String queryId, CountDownLatch threadLatch,
+      Map<Thread, CPUMemThreadLevelAccountingObjects.ThreadEntry> 
threadEntries) {
+    TaskThread
+        anchorThread = getTaskThread(queryId, 
CommonConstants.Accounting.ANCHOR_TASK_ID, threadLatch, null);
+    threadEntries.put(anchorThread._workerThread, anchorThread._threadEntry);
+    anchorThread._threadEntry._currentThreadMemoryAllocationSampleBytes = 1000;
+
+    CPUMemThreadLevelAccountingObjects.ThreadEntry anchorEntry = new 
CPUMemThreadLevelAccountingObjects.ThreadEntry();
+    anchorEntry._currentThreadTaskStatus.set(
+        new CPUMemThreadLevelAccountingObjects.TaskEntry(queryId, 
CommonConstants.Accounting.ANCHOR_TASK_ID,
+            ThreadExecutionContext.TaskType.SSE, anchorThread._workerThread));
+    anchorEntry._currentThreadMemoryAllocationSampleBytes = 1000;
+    threadEntries.put(anchorThread._workerThread, anchorEntry);

Review Comment:
   This `put` overwrites the earlier entry for the same 
`anchorThread._workerThread`, discarding the first `ThreadEntry`. Consider 
removing the first or second insertion or use distinct keys so both entries are 
preserved as intended.
   ```suggestion
       threadEntries.put(anchorThread._workerThread + "_anchorEntry", 
anchorEntry);
   ```



##########
pinot-core/src/main/java/org/apache/pinot/core/accounting/PerQueryCPUMemAccountantFactory.java:
##########
@@ -870,23 +894,29 @@ private void killMostExpensiveQuery() {
         }
         AggregatedStats maxUsageTuple;
         if (_isThreadMemorySamplingEnabled) {
-          maxUsageTuple = 
Collections.max(_aggregatedUsagePerActiveQuery.values(),
-              Comparator.comparing(AggregatedStats::getAllocatedBytes));
-          boolean shouldKill = config.isOomKillQueryEnabled()
-              && maxUsageTuple._allocatedBytes > 
config.getMinMemoryFootprintForKill();
-          if (shouldKill) {
-            maxUsageTuple._exceptionAtomicReference
-                .set(new RuntimeException(String.format(
-                    " Query %s got killed because using %d bytes of memory on 
%s: %s, exceeding the quota",
-                    maxUsageTuple._queryId, maxUsageTuple.getAllocatedBytes(), 
_instanceType, _instanceId)));
-            interruptRunnerThread(maxUsageTuple.getAnchorThread());
-            logTerminatedQuery(maxUsageTuple, _usedBytes);
-          } else if (!config.isOomKillQueryEnabled()) {
-            LOGGER.warn("Query {} got picked because using {} bytes of memory, 
actual kill committed false "
-                    + "because oomKillQueryEnabled is false",
-                maxUsageTuple._queryId, maxUsageTuple._allocatedBytes);
+          maxUsageTuple = _aggregatedUsagePerActiveQuery.values().stream()
+              .filter(stats -> 
!_cancelSentQueries.contains(stats.getQueryId()))
+              .max(Comparator.comparing(AggregatedStats::getAllocatedBytes))
+              .orElse(null);
+          if (maxUsageTuple != null) {
+            boolean shouldKill = config.isOomKillQueryEnabled()
+                && maxUsageTuple._allocatedBytes > 
config.getMinMemoryFootprintForKill();
+            if (shouldKill) {
+              maxUsageTuple._exceptionAtomicReference
+                  .set(new RuntimeException(String.format(
+                      " Query %s got killed because using %d bytes of memory 
on %s: %s, exceeding the quota",
+                      maxUsageTuple._queryId, 
maxUsageTuple.getAllocatedBytes(), _instanceType, _instanceId)));
+              interruptRunnerThread(maxUsageTuple);
+              logTerminatedQuery(maxUsageTuple, _usedBytes);
+            } else if (!config.isOomKillQueryEnabled()) {
+              LOGGER.warn("Query {} got picked because using {} bytes of 
memory, actual kill committed false "
+                      + "because oomKillQueryEnabled is false",
+                  maxUsageTuple._queryId, maxUsageTuple._allocatedBytes);
+            } else {
+              LOGGER.warn("But all queries are below quota, no query killed");
+            }
           } else {
-            LOGGER.warn("But all queries are below quota, no query killed");
+            LOGGER.warn("No query found to kill based on memory usage");

Review Comment:
   [nitpick] Logging a missing eligible query at WARN level may noise up logs 
in normal operation cycles when cancellations are filtered out. Consider 
changing this to DEBUG or INFO.
   ```suggestion
               LOGGER.debug("No query found to kill based on memory usage");
   ```



-- 
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]

Reply via email to