cryptoe commented on code in PR #13062:
URL: https://github.com/apache/druid/pull/13062#discussion_r997284741
##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/error/MSQErrorReport.java:
##########
@@ -36,6 +36,8 @@
{
private final String taskId;
@Nullable
+ private final Integer workerNumber;
Review Comment:
Is this still being used ?
##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/DurableStorageOutputChannelFactory.java:
##########
@@ -112,31 +133,28 @@ public OutputChannel openNilChannel(int partitionNumber)
throw new ISE(
e,
"Unable to create empty remote output of workerTask[%s] stage[%d]
partition[%d]",
- workerTaskId,
+ workerNumber,
stageNumber,
partitionNumber
);
}
}
- public static String getControllerDirectory(final String controllerTaskId)
- {
- return StringUtils.format("controller_%s", IdUtils.validateId("controller
task ID", controllerTaskId));
- }
-
- public static String getPartitionFileName(
- final String controllerTaskId,
- final String workerTaskId,
- final int stageNumber,
- final int partitionNumber
- )
+ /**
+ * Creates a file with name __success and adds the worker's id which has
successfully written its outputs. While reading
+ * this file can be used to find out the worker which has written its
outputs completely.
+ * Rename operation is not very quick in cloud storage like S3 due to which
this alternative
+ * route has been taken
+ */
+ public void createSuccessFile(String taskId) throws IOException
Review Comment:
We should also mention that if the path already exists this operation is a
no op.
##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/DurableStorageInputChannelFactory.java:
##########
@@ -112,11 +118,58 @@ 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);
+ }
+
Review Comment:
Let's debug log the successfull taskId ?
--
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]