You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by GitBox <gi...@apache.org> on 2022/09/14 11:48:39 UTC

[GitHub] [druid] cryptoe commented on a diff in pull request #13062: Use worker number instead of task id in MSQ for communication to/from workers.

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


##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java:
##########
@@ -856,7 +874,37 @@ public void onSuccess(final List<Object> workerResultAndOutputChannelsResolved)
             for (OutputChannel channel : outputChannels.getAllChannels()) {
               stageOutputs.computeIfAbsent(stageDef.getId(), ignored1 -> new ConcurrentHashMap<>())
                           .computeIfAbsent(channel.getPartitionNumber(), ignored2 -> channel.getReadableChannel());
+
+              if (durableStageStorageEnabled) {
+                // Mark the stuff as __success there

Review Comment:
   Nit: Lets improve this comment. 
   We want to be sure that we only read when the write is fully finished hence add a SUCCESS file 



##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/DurableStorageInputChannelFactory.java:
##########
@@ -112,11 +119,33 @@ 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
       );
     }
   }
+
+  @Nullable
+  public String findSuccessfulPartitionOutput(
+      final String controllerTaskId,
+      final int workerNo,
+      final int stageNumber,
+      final int partitionNumber
+  ) throws IOException
+  {
+    List<String> fileNames = storageConnector.lsFiles(
+        DurableStorageOutputChannelFactory.getPartitionOutputsFolderName(
+            controllerTaskId,
+            workerNo,
+            stageNumber,
+            partitionNumber
+        )
+    );
+    Optional<String> maybeFileName = fileNames.stream()

Review Comment:
   So we can have issues here. 
   This logic is not deterministic. 
   Lets say we have a case where one zombie task also wrote its output successfully to durable storage. This means, the output of that task will also be considered to be read 
   So now you have 
   `/xxx/workerNumber/task_good_id/part_1_success`
   `/xxx/workerNumber/task_zombie_id/part_1_success`
   
   As rows in both of them may not follow the same order we might be in a soup if worker one reads zombie_task files and worker 2 reads good_id files. 
   
   
   The best fix would be to figure out which path to read the data from at one place IE in the controller. 
   
   If we are not able to do that then we have to make some changes into how the correct taskID files are picked. 
   
   I suggest using time as the tiebreaker. Which ever successful file is written first, we use that. Basically, first write wins. 
   
   
   
   cc @gianm 
   



##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/DurableStorageOutputChannelFactory.java:
##########
@@ -112,31 +132,52 @@ 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 void markAsSuccess(int partitionNumber) throws IOException
+  {
+    String oldPath = getPartitionOutputFileNameForTask(controllerTaskId, workerNumber, stageNumber, partitionNumber, taskId);
+    storageConnector.moveFile(oldPath, StringUtils.format("%s%s", oldPath, SUCCESSFUL_SUFFIX));

Review Comment:
   That way we can also use the timestamp of that file for tiebreaker. 



##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/DurableStorageOutputChannelFactory.java:
##########
@@ -112,31 +132,52 @@ 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 void markAsSuccess(int partitionNumber) throws IOException
+  {
+    String oldPath = getPartitionOutputFileNameForTask(controllerTaskId, workerNumber, stageNumber, partitionNumber, taskId);
+    storageConnector.moveFile(oldPath, StringUtils.format("%s%s", oldPath, SUCCESSFUL_SUFFIX));

Review Comment:
   This is a very expensive call in s3. IIRC you basically copy from old path to new path. Its not just a metadata update as in the case of HDFS. 
   
   IMHO we might need a better approach here. 
   
   Just create a __SUCCESS file in the dir ?



##########
extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java:
##########
@@ -856,7 +874,37 @@ public void onSuccess(final List<Object> workerResultAndOutputChannelsResolved)
             for (OutputChannel channel : outputChannels.getAllChannels()) {
               stageOutputs.computeIfAbsent(stageDef.getId(), ignored1 -> new ConcurrentHashMap<>())
                           .computeIfAbsent(channel.getPartitionNumber(), ignored2 -> channel.getReadableChannel());
+
+              if (durableStageStorageEnabled) {
+                // Mark the stuff as __success there
+                DurableStorageOutputChannelFactory durableStorageOutputChannelFactory =
+                    DurableStorageOutputChannelFactory.createStandardImplementation(
+                        task.getControllerTaskId(),
+                        task().getWorkerNumber(),
+                        stageDef.getStageNumber(),
+                        task().getId(),
+                        frameContext.memoryParameters().getStandardFrameSize(),
+                        MSQTasks.makeStorageConnector(context.injector())
+                    );
+                try {
+                  durableStorageOutputChannelFactory.markAsSuccess(channel.getPartitionNumber());

Review Comment:
   Looks like you are marking the file successfull twice accidentally. 



-- 
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-unsubscribe@druid.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org