cryptoe commented on code in PR #13062:
URL: https://github.com/apache/druid/pull/13062#discussion_r985391069
##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/DurableStorageInputChannelFactory.java:
##########
@@ -112,11 +118,59 @@ 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)) {
+ // TODO: Check if this call can block indefinitely
+ successfulTaskId = IOUtils.toString(is, StandardCharsets.UTF_8);
Review Comment:
I think this should be fine. As the success file would be less than 100
bytes, we should be cool with it.
##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerClient.java:
##########
@@ -72,7 +72,7 @@ ListenableFuture<Void> postResultPartitionBoundaries(
* kind of unrecoverable exception).
*/
ListenableFuture<Boolean> fetchChannelData(
- String workerTaskId,
+ int workerNumber,
Review Comment:
IMO, the logic of which worker task to contact should be built outside this
client as we would never want half the requests going to older workers, when
worker fault tolerance is there, and half of them to the new workers.
Hence, I think we should not touch
`extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerClient.java`
this class.
Thoughts?
##########
extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java:
##########
@@ -1086,6 +1091,65 @@ public void testGroupByMultiValueMeasureQuery()
.verifyResults();
}
+ @Test
+ public void testGroupByOnFooWithDurableStoragePathAssertions()
+ {
+ RowSignature rowSignature = RowSignature.builder()
+ .add("cnt", ColumnType.LONG)
+ .add("cnt1", ColumnType.LONG)
+ .build();
+
+ ExecutorService executorService = Execs.singleThreaded("path-verifier");
+ final AtomicBoolean existsOnce = new AtomicBoolean(false);
+ executorService.submit(() -> {
+ while (true) {
+ File successFile = new File(
Review Comment:
This logic is very racy. Can we intercept calls on the local file storage
connector and then assert on the paths ?
--
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]