richardstartin commented on a change in pull request #7893:
URL: https://github.com/apache/pinot/pull/7893#discussion_r767761132



##########
File path: 
pinot-server/src/main/java/org/apache/pinot/server/starter/helix/HelixInstanceDataManager.java
##########
@@ -212,34 +217,53 @@ public void reloadSegment(String tableNameWithType, 
String segmentName, boolean
   }
 
   @Override
-  public void reloadAllSegments(String tableNameWithType, boolean 
forceDownload) {
+  public void reloadAllSegments(String tableNameWithType, boolean 
forceDownload, int parallelism)
+      throws Exception {
     LOGGER.info("Reloading all segments in table: {}", tableNameWithType);
     TableConfig tableConfig = 
ZKMetadataProvider.getTableConfig(_propertyStore, tableNameWithType);
     Preconditions.checkNotNull(tableConfig);
 
     Schema schema = ZKMetadataProvider.getTableSchema(_propertyStore, 
tableNameWithType);
 
     List<String> failedSegments = new ArrayList<>();
-    Exception sampleException = null;
+
     List<SegmentMetadata> segmentsMetadata = 
getAllSegmentsMetadata(tableNameWithType);
+
+    int nosWorkers = Math.min(segmentsMetadata.size(), parallelism);
+
+    ExecutorService workers = Executors.newFixedThreadPool(nosWorkers);
+
+    final AtomicReference<Exception> sampleException = new AtomicReference<>();
+
+    CountDownLatch latch = new CountDownLatch(nosWorkers);
+
     for (SegmentMetadata segmentMetadata : segmentsMetadata) {
-      try {
-        reloadSegment(tableNameWithType, segmentMetadata, tableConfig, schema, 
forceDownload);
-      } catch (Exception e) {
-        String segmentName = segmentMetadata.getName();
-        LOGGER.error("Caught exception while reloading segment: {} in table: 
{}", segmentName, tableNameWithType, e);
-        failedSegments.add(segmentName);
-        sampleException = e;
-      }
+      CompletableFuture.runAsync(() -> {
+        try {
+          reloadSegment(tableNameWithType, segmentMetadata, tableConfig, 
schema, forceDownload);
+        } catch (Exception e) {
+          String segmentName = segmentMetadata.getName();
+          LOGGER.error("Caught exception while reloading segment: {} in table: 
{}", segmentName, tableNameWithType, e);
+          failedSegments.add(segmentName);
+          sampleException.set(e);
+        } finally {
+          latch.countDown();
+        }
+      }, workers);

Review comment:
       I think you can make better use of the `CompletableFuture` API here:
   
   ```java
   CompletableFuture.allOf(segmentsMetadata.stream()
   .map(segmentMetadata -> CompletableFuture.runAsync(() -> {... reload the 
segment ...}, workers)
   .toArray(CompletableFuture[]::new))
   .get(timeout, TimeUnit.MILLISECONDS);
   ```
   
   Which would eliminate the latch and make the code easier to reason about.




-- 
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: commits-unsubscr...@pinot.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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

Reply via email to