This is an automated email from the ASF dual-hosted git repository.

somandal pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/pinot.git


The following commit(s) were added to refs/heads/master by this push:
     new 744199f9cec Improve SEGMENT_TABLE_DOWNLOAD_COUNT accuracy (#16342)
744199f9cec is described below

commit 744199f9cec04e2b82fac475561504dde7e24fdc
Author: Songqiao Su <andysongq...@gmail.com>
AuthorDate: Mon Jul 14 19:39:48 2025 -0700

    Improve SEGMENT_TABLE_DOWNLOAD_COUNT accuracy (#16342)
---
 .../pinot/core/data/manager/BaseTableDataManager.java       | 13 ++++++-------
 1 file changed, 6 insertions(+), 7 deletions(-)

diff --git 
a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java
 
b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java
index dd35dcf7c10..b5495b57232 100644
--- 
a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java
+++ 
b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/BaseTableDataManager.java
@@ -216,18 +216,17 @@ public abstract class BaseTableDataManager implements 
TableDataManager {
           _streamSegmentDownloadUntarRateLimitBytesPerSec);
     }
     int maxParallelSegmentDownloads = 
instanceDataManagerConfig.getMaxParallelSegmentDownloads();
+    _numSegmentsAcquiredDownloadSemaphore = new AtomicInteger(0);
+    _serverMetrics.setValueOfTableGauge(_tableNameWithType, 
ServerGauge.SEGMENT_TABLE_DOWNLOAD_COUNT, 0);
     if (maxParallelSegmentDownloads > 0) {
       LOGGER.info(
           "Construct segment download semaphore for Table: {}. Maximum number 
of parallel segment downloads: {}",
           _tableNameWithType, maxParallelSegmentDownloads);
       _segmentDownloadSemaphore = new Semaphore(maxParallelSegmentDownloads, 
true);
-      _numSegmentsAcquiredDownloadSemaphore = new AtomicInteger(0);
       _serverMetrics.setValueOfTableGauge(_tableNameWithType, 
ServerGauge.SEGMENT_TABLE_DOWNLOAD_THROTTLE_THRESHOLD,
           maxParallelSegmentDownloads);
-      _serverMetrics.setValueOfTableGauge(_tableNameWithType, 
ServerGauge.SEGMENT_TABLE_DOWNLOAD_COUNT, 0);
     } else {
       _segmentDownloadSemaphore = null;
-      _numSegmentsAcquiredDownloadSemaphore = null;
     }
     _logger = LoggerFactory.getLogger(_tableNameWithType + "-" + 
getClass().getSimpleName());
 
@@ -946,8 +945,6 @@ public abstract class BaseTableDataManager implements 
TableDataManager {
       _logger.info("Acquiring table level segment download semaphore for 
segment: {}, queue-length: {} ", segmentName,
           _segmentDownloadSemaphore.getQueueLength());
       _segmentDownloadSemaphore.acquire();
-      _serverMetrics.setValueOfTableGauge(_tableNameWithType, 
ServerGauge.SEGMENT_TABLE_DOWNLOAD_COUNT,
-          _numSegmentsAcquiredDownloadSemaphore.incrementAndGet());
       _logger.info("Acquired table level segment download semaphore for 
segment: {} (lock-time={}ms, queue-length={}).",
           segmentName, System.currentTimeMillis() - startTime, 
_segmentDownloadSemaphore.getQueueLength());
     }
@@ -962,6 +959,8 @@ public abstract class BaseTableDataManager implements 
TableDataManager {
                 + "queue-length={}).", segmentName, System.currentTimeMillis() 
- startTime,
             segmentDownloadThrottler.getQueueLength());
       }
+      _serverMetrics.setValueOfTableGauge(_tableNameWithType, 
ServerGauge.SEGMENT_TABLE_DOWNLOAD_COUNT,
+          _numSegmentsAcquiredDownloadSemaphore.incrementAndGet());
       try {
         File untarredSegmentDir;
         if (_isStreamSegmentDownloadUntar && zkMetadata.getCrypterName() == 
null) {
@@ -999,9 +998,9 @@ public abstract class BaseTableDataManager implements 
TableDataManager {
     } finally {
       if (_segmentDownloadSemaphore != null) {
         _segmentDownloadSemaphore.release();
-        _serverMetrics.setValueOfTableGauge(_tableNameWithType, 
ServerGauge.SEGMENT_TABLE_DOWNLOAD_COUNT,
-            _numSegmentsAcquiredDownloadSemaphore.decrementAndGet());
       }
+      _serverMetrics.setValueOfTableGauge(_tableNameWithType, 
ServerGauge.SEGMENT_TABLE_DOWNLOAD_COUNT,
+          _numSegmentsAcquiredDownloadSemaphore.decrementAndGet());
     }
   }
 


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org
For additional commands, e-mail: commits-h...@pinot.apache.org

Reply via email to