cryptoe commented on code in PR #13062:
URL: https://github.com/apache/druid/pull/13062#discussion_r1004188553


##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/MSQWarningReportSimplePublisher.java:
##########
@@ -32,19 +32,22 @@ public class MSQWarningReportSimplePublisher implements 
MSQWarningReportPublishe
 {
 
   final String workerId;
+  final int workerNumber;

Review Comment:
   do we still need workerID and workerNumber ?



##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/DurableStorageInputChannelFactory.java:
##########
@@ -112,11 +118,64 @@ public ReadableFrameChannel openChannel(StageId stageId, 
int workerNumber, int p
     catch (Exception e) {
       throw new IOE(
           e,
-          "Could not find remote output of worker task[%s] stage[%d] 
partition[%d]",
-          workerTaskId,
+          "Could not find remote output of worker task[%d] stage[%d] 
partition[%d]",
+          workerNumber,
           stageId.getStageNumber(),
           partitionNumber
       );
     }
   }
+
+  /**
+   * Given an input worker number, stage number and the partition number, this 
method figures out the exact location
+   * where the outputs would be present in the durable storage and returns the 
complete path or throws an exception
+   * if no such file exists in the durable storage
+   * More information at {@link 
DurableStorageOutputChannelFactory#createSuccessFile(String)}
+   */
+  public String findSuccessfulPartitionOutput(
+      final String controllerTaskId,
+      final int workerNo,
+      final int stageNumber,
+      final int partitionNumber
+  ) throws IOException
+  {
+    String successfulFilePath = DurableStorageUtils.getSuccessFilePath(
+        controllerTaskId,
+        stageNumber,
+        workerNo
+    );
+
+    if (!storageConnector.pathExists(successfulFilePath)) {
+      throw new ISE(
+          "No file present at the location [%s]. Unable to read the inputs for 
worker: [%d], stage: [%d], partition: [%d]",

Review Comment:
   ```suggestion
             "No file present at the location [%s]. Unable to read the output 
of worker: [%d], stage: [%d], partition: [%d]",
   ```



##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/DurableStorageInputChannelFactory.java:
##########
@@ -112,11 +118,64 @@ public ReadableFrameChannel openChannel(StageId stageId, 
int workerNumber, int p
     catch (Exception e) {
       throw new IOE(
           e,
-          "Could not find remote output of worker task[%s] stage[%d] 
partition[%d]",
-          workerTaskId,
+          "Could not find remote output of worker task[%d] stage[%d] 
partition[%d]",
+          workerNumber,
           stageId.getStageNumber(),
           partitionNumber
       );
     }
   }
+
+  /**
+   * Given an input worker number, stage number and the partition number, this 
method figures out the exact location
+   * where the outputs would be present in the durable storage and returns the 
complete path or throws an exception
+   * if no such file exists in the durable storage
+   * More information at {@link 
DurableStorageOutputChannelFactory#createSuccessFile(String)}
+   */
+  public String findSuccessfulPartitionOutput(
+      final String controllerTaskId,
+      final int workerNo,
+      final int stageNumber,
+      final int partitionNumber
+  ) throws IOException
+  {
+    String successfulFilePath = DurableStorageUtils.getSuccessFilePath(
+        controllerTaskId,
+        stageNumber,
+        workerNo
+    );
+
+    if (!storageConnector.pathExists(successfulFilePath)) {
+      throw new ISE(
+          "No file present at the location [%s]. Unable to read the inputs for 
worker: [%d], stage: [%d], partition: [%d]",
+          successfulFilePath,
+          workerNo,
+          stageNumber,
+          partitionNumber
+      );
+    }
+
+    String successfulTaskId;
+
+    try (InputStream is = storageConnector.read(successfulFilePath)) {
+      successfulTaskId = IOUtils.toString(is, StandardCharsets.UTF_8);
+    }
+    if (successfulTaskId == null) {
+      throw new ISE("Unable to read the task id from the file: [%s]", 
successfulFilePath);
+    }
+    LOG.info(
+        "Successful task id for stage [%d] and partition [%d]: [%s]",
+        stageNumber,
+        partitionNumber,
+        successfulTaskId
+    );

Review Comment:
   ```suggestion
       LOG.debug(
           "Reading output of stage [%d] and partition [%d] from task id [%s]", 
           stageNumber,
           partitionNumber,
           successfulTaskId
       );
   ```



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