FrankChen021 commented on code in PR #18489:
URL: https://github.com/apache/druid/pull/18489#discussion_r2451628475


##########
server/src/main/java/org/apache/druid/segment/loading/SegmentLocalCacheManager.java:
##########
@@ -190,22 +193,47 @@ public List<DataSegment> getCachedSegments() throws 
IOException
       );
     }
 
-    final List<DataSegment> cachedSegments = new ArrayList<>();
     final File[] segmentsToLoad = retrieveSegmentMetadataFiles();
-
+    final ConcurrentLinkedQueue<DataSegment> cachedSegments = new 
ConcurrentLinkedQueue<>();
     AtomicInteger ignoredFilesCounter = new AtomicInteger(0);
+    CountDownLatch latch = new CountDownLatch(segmentsToLoad.length);
 
-    for (int i = 0; i < segmentsToLoad.length; i++) {
-      final File file = segmentsToLoad[i];
-      log.info("Loading segment cache file [%d/%d][%s].", i + 1, 
segmentsToLoad.length, file);
-      try {
-        addFilesToCachedSegments(file, ignoredFilesCounter, cachedSegments);
-      }
-      catch (Exception e) {
-        log.makeAlert(e, "Failed to load segment from segment cache file.")
-           .addData("file", file)
-           .emit();
-      }
+    boolean createdNewExecutorServiceToLoadSegmentCache = loadOnBootstrapExec 
== null;
+    ExecutorService executorService = 
createdNewExecutorServiceToLoadSegmentCache
+                                      ? 
MoreExecutors.newDirectExecutorService()
+                                      : loadOnBootstrapExec;
+
+    Stopwatch stopwatch = Stopwatch.createStarted();
+    log.info("Retrieving [%d] cached segment metadata files to cache.", 
segmentsToLoad.length);
+
+    for (File file : segmentsToLoad) {
+      executorService.submit(() -> {
+        try {
+          loadToCachedSegmentsFromFile(cachedSegments, file, 
ignoredFilesCounter);
+        }
+        catch (Exception e) {
+          log.makeAlert(e, "Failed to load segment from segment cache file.")
+             .addData("file", file)
+             .emit();
+        }
+
+        latch.countDown();

Review Comment:
   put in the finally block



##########
server/src/main/java/org/apache/druid/segment/loading/SegmentLocalCacheManager.java:
##########
@@ -190,22 +193,47 @@ public List<DataSegment> getCachedSegments() throws 
IOException
       );
     }
 
-    final List<DataSegment> cachedSegments = new ArrayList<>();
     final File[] segmentsToLoad = retrieveSegmentMetadataFiles();
-
+    final ConcurrentLinkedQueue<DataSegment> cachedSegments = new 
ConcurrentLinkedQueue<>();
     AtomicInteger ignoredFilesCounter = new AtomicInteger(0);
+    CountDownLatch latch = new CountDownLatch(segmentsToLoad.length);
 
-    for (int i = 0; i < segmentsToLoad.length; i++) {
-      final File file = segmentsToLoad[i];
-      log.info("Loading segment cache file [%d/%d][%s].", i + 1, 
segmentsToLoad.length, file);
-      try {
-        addFilesToCachedSegments(file, ignoredFilesCounter, cachedSegments);
-      }
-      catch (Exception e) {
-        log.makeAlert(e, "Failed to load segment from segment cache file.")
-           .addData("file", file)
-           .emit();
-      }
+    boolean createdNewExecutorServiceToLoadSegmentCache = loadOnBootstrapExec 
== null;
+    ExecutorService executorService = 
createdNewExecutorServiceToLoadSegmentCache
+                                      ? 
MoreExecutors.newDirectExecutorService()
+                                      : loadOnBootstrapExec;
+
+    Stopwatch stopwatch = Stopwatch.createStarted();
+    log.info("Retrieving [%d] cached segment metadata files to cache.", 
segmentsToLoad.length);
+
+    for (File file : segmentsToLoad) {
+      executorService.submit(() -> {
+        try {
+          loadToCachedSegmentsFromFile(cachedSegments, file, 
ignoredFilesCounter);
+        }
+        catch (Exception e) {
+          log.makeAlert(e, "Failed to load segment from segment cache file.")
+             .addData("file", file)
+             .emit();
+        }
+
+        latch.countDown();
+      });
+    }
+
+    try {
+      latch.await();
+    }
+    catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      log.makeAlert(e, "Interrupted when trying to retrieve cached segment 
metadata files");

Review Comment:
   I think there's no need to send an alert for this exception



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