mirror of https://github.com/apache/druid.git
Use worker number instead of task id in MSQ for communication to/from workers. (#13062)
* Conversion from taskId to workerNumber in the workerClient * storage connector changes, suffix file when finish writing to it * Fix tests * Trigger Build * convert IntFunction to a dedicated interface * first review round * use a dummy file to indicate success * fetch the first filename from the list in case of multiple files * tests working, fix semantic issue with ls * change how the success flag works * comments, checkstyle, method rename * fix test * forbiddenapis fix * Trigger Build * change the writer * dead store fix * Review comments * revert changes * review * review comments * Update extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/DurableStorageInputChannelFactory.java Co-authored-by: Karan Kumar <karankumar1100@gmail.com> * Update extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/DurableStorageInputChannelFactory.java Co-authored-by: Karan Kumar <karankumar1100@gmail.com> * update error messages * better error messages * fix checkstyle Co-authored-by: Karan Kumar <karankumar1100@gmail.com>
This commit is contained in:
parent
018f984781
commit
7cb21cb968
|
@ -31,7 +31,7 @@ import java.io.OutputStream;
|
|||
import java.nio.file.Files;
|
||||
|
||||
/**
|
||||
* Implementation that uses local filesystem. All paths are appended with the base path, in such a way that its not visible
|
||||
* Implementation that uses local filesystem. All paths are appended with the base path, in such a way that it is not visible
|
||||
* to the users of this class.
|
||||
*/
|
||||
public class LocalFileStorageConnector implements StorageConnector
|
||||
|
@ -54,10 +54,6 @@ public class LocalFileStorageConnector implements StorageConnector
|
|||
/**
|
||||
* Reads the file present as basePath + path. Will throw an IO exception in case the file is not present.
|
||||
* Closing of the stream is the responsibility of the caller.
|
||||
*
|
||||
* @param path
|
||||
* @return
|
||||
* @throws IOException
|
||||
*/
|
||||
@Override
|
||||
public InputStream read(String path) throws IOException
|
||||
|
|
|
@ -71,7 +71,6 @@ public interface ControllerClient extends AutoCloseable
|
|||
* Client side method to inform the controller about the warnings generated by the given worker.
|
||||
*/
|
||||
void postWorkerWarning(
|
||||
String workerId,
|
||||
List<MSQErrorReport> MSQErrorReports
|
||||
) throws IOException;
|
||||
List<String> getTaskList() throws IOException;
|
||||
|
|
|
@ -74,7 +74,6 @@ public interface ControllerContext
|
|||
* Client for communicating with workers.
|
||||
*/
|
||||
WorkerClient taskClientFor(Controller controller);
|
||||
|
||||
/**
|
||||
* Writes controller task report.
|
||||
*/
|
||||
|
|
|
@ -146,7 +146,7 @@ import org.apache.druid.msq.querykit.ShuffleSpecFactory;
|
|||
import org.apache.druid.msq.querykit.groupby.GroupByQueryKit;
|
||||
import org.apache.druid.msq.querykit.scan.ScanQueryKit;
|
||||
import org.apache.druid.msq.shuffle.DurableStorageInputChannelFactory;
|
||||
import org.apache.druid.msq.shuffle.DurableStorageOutputChannelFactory;
|
||||
import org.apache.druid.msq.shuffle.DurableStorageUtils;
|
||||
import org.apache.druid.msq.shuffle.WorkerInputChannelFactory;
|
||||
import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot;
|
||||
import org.apache.druid.msq.util.DimensionSchemaUtils;
|
||||
|
@ -1202,7 +1202,6 @@ public class ControllerImpl implements Controller
|
|||
if (MultiStageQueryContext.isDurableStorageEnabled(task.getQuerySpec().getQuery().context())) {
|
||||
inputChannelFactory = DurableStorageInputChannelFactory.createStandardImplementation(
|
||||
id(),
|
||||
() -> taskIds,
|
||||
MSQTasks.makeStorageConnector(context.injector()),
|
||||
closer
|
||||
);
|
||||
|
@ -1303,7 +1302,7 @@ public class ControllerImpl implements Controller
|
|||
private void cleanUpDurableStorageIfNeeded()
|
||||
{
|
||||
if (MultiStageQueryContext.isDurableStorageEnabled(task.getQuerySpec().getQuery().context())) {
|
||||
final String controllerDirName = DurableStorageOutputChannelFactory.getControllerDirectory(task.getId());
|
||||
final String controllerDirName = DurableStorageUtils.getControllerDirectory(task.getId());
|
||||
try {
|
||||
// Delete all temporary files as a failsafe
|
||||
MSQTasks.makeStorageConnector(context.injector()).deleteRecursively(controllerDirName);
|
||||
|
|
|
@ -36,7 +36,7 @@ public interface WorkerClient extends AutoCloseable
|
|||
/**
|
||||
* Worker's client method to add a {@link WorkOrder} to the worker to work on
|
||||
*/
|
||||
ListenableFuture<Void> postWorkOrder(String workerId, WorkOrder workOrder);
|
||||
ListenableFuture<Void> postWorkOrder(String workerTaskId, WorkOrder workOrder);
|
||||
|
||||
/**
|
||||
* Worker's client method to inform it of the partition boundaries for the given stage. This is usually invoked by the
|
||||
|
@ -50,13 +50,15 @@ public interface WorkerClient extends AutoCloseable
|
|||
|
||||
/**
|
||||
* Worker's client method to inform that the work has been done, and it can initiate cleanup and shutdown
|
||||
* @param workerTaskId
|
||||
*/
|
||||
ListenableFuture<Void> postFinish(String workerId);
|
||||
ListenableFuture<Void> postFinish(String workerTaskId);
|
||||
|
||||
/**
|
||||
* Fetches all the counters gathered by that worker
|
||||
* @param workerTaskId
|
||||
*/
|
||||
ListenableFuture<CounterSnapshotsTree> getCounters(String workerId);
|
||||
ListenableFuture<CounterSnapshotsTree> getCounters(String workerTaskId);
|
||||
|
||||
/**
|
||||
* Worker's client method that informs it that the results and resources for the given stage are no longer required
|
||||
|
|
|
@ -102,6 +102,7 @@ import org.apache.druid.msq.kernel.worker.WorkerStagePhase;
|
|||
import org.apache.druid.msq.querykit.DataSegmentProvider;
|
||||
import org.apache.druid.msq.shuffle.DurableStorageInputChannelFactory;
|
||||
import org.apache.druid.msq.shuffle.DurableStorageOutputChannelFactory;
|
||||
import org.apache.druid.msq.shuffle.DurableStorageUtils;
|
||||
import org.apache.druid.msq.shuffle.WorkerInputChannelFactory;
|
||||
import org.apache.druid.msq.statistics.ClusterByStatisticsCollector;
|
||||
import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot;
|
||||
|
@ -210,7 +211,12 @@ public class WorkerImpl implements Worker
|
|||
}
|
||||
catch (Throwable e) {
|
||||
maybeErrorReport = Optional.of(
|
||||
MSQErrorReport.fromException(id(), MSQTasks.getHostFromSelfNode(selfDruidNode), null, e)
|
||||
MSQErrorReport.fromException(
|
||||
id(),
|
||||
MSQTasks.getHostFromSelfNode(selfDruidNode),
|
||||
null,
|
||||
e
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -241,6 +247,7 @@ public class WorkerImpl implements Worker
|
|||
this.controllerClient = context.makeControllerClient(task.getControllerTaskId());
|
||||
closer.register(controllerClient::close);
|
||||
context.registerWorker(this, closer); // Uses controllerClient, so must be called after that is initialized
|
||||
|
||||
this.workerClient = new ExceptionWrappingWorkerClient(context.makeWorkerClient());
|
||||
closer.register(workerClient::close);
|
||||
|
||||
|
@ -583,7 +590,6 @@ public class WorkerImpl implements Worker
|
|||
if (durableStageStorageEnabled) {
|
||||
return DurableStorageInputChannelFactory.createStandardImplementation(
|
||||
task.getControllerTaskId(),
|
||||
workerTaskList,
|
||||
MSQTasks.makeStorageConnector(context.injector()),
|
||||
closer
|
||||
);
|
||||
|
@ -601,8 +607,9 @@ public class WorkerImpl implements Worker
|
|||
if (durableStageStorageEnabled) {
|
||||
return DurableStorageOutputChannelFactory.createStandardImplementation(
|
||||
task.getControllerTaskId(),
|
||||
id(),
|
||||
task().getWorkerNumber(),
|
||||
stageNumber,
|
||||
task().getId(),
|
||||
frameSize,
|
||||
MSQTasks.makeStorageConnector(context.injector())
|
||||
);
|
||||
|
@ -709,18 +716,18 @@ public class WorkerImpl implements Worker
|
|||
// Therefore, the logic for cleaning the stage output in case of a worker/machine crash has to be external.
|
||||
// We currently take care of this in the controller.
|
||||
if (durableStageStorageEnabled) {
|
||||
final String fileName = DurableStorageOutputChannelFactory.getPartitionFileName(
|
||||
final String folderName = DurableStorageUtils.getTaskIdOutputsFolderName(
|
||||
task.getControllerTaskId(),
|
||||
task.getId(),
|
||||
stageId.getStageNumber(),
|
||||
partition
|
||||
task.getWorkerNumber(),
|
||||
task.getId()
|
||||
);
|
||||
try {
|
||||
MSQTasks.makeStorageConnector(context.injector()).deleteFile(fileName);
|
||||
MSQTasks.makeStorageConnector(context.injector()).deleteRecursively(folderName);
|
||||
}
|
||||
catch (Exception e) {
|
||||
// If an error is thrown while cleaning up a file, log it and try to continue with the cleanup
|
||||
log.warn(e, "Error while cleaning up temporary files at path " + fileName);
|
||||
log.warn(e, "Error while cleaning up folder at path " + folderName);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -888,7 +895,38 @@ public class WorkerImpl implements Worker
|
|||
for (OutputChannel channel : outputChannels.getAllChannels()) {
|
||||
stageOutputs.computeIfAbsent(stageDef.getId(), ignored1 -> new ConcurrentHashMap<>())
|
||||
.computeIfAbsent(channel.getPartitionNumber(), ignored2 -> channel.getReadableChannel());
|
||||
|
||||
}
|
||||
|
||||
if (durableStageStorageEnabled) {
|
||||
// Once the outputs channels have been resolved and are ready for reading, the worker appends the filename
|
||||
// with a special marker flag and adds it to the
|
||||
DurableStorageOutputChannelFactory durableStorageOutputChannelFactory =
|
||||
DurableStorageOutputChannelFactory.createStandardImplementation(
|
||||
task.getControllerTaskId(),
|
||||
task().getWorkerNumber(),
|
||||
stageDef.getStageNumber(),
|
||||
task().getId(),
|
||||
frameContext.memoryParameters().getStandardFrameSize(),
|
||||
MSQTasks.makeStorageConnector(context.injector())
|
||||
);
|
||||
try {
|
||||
durableStorageOutputChannelFactory.createSuccessFile(task.getId());
|
||||
}
|
||||
catch (IOException e) {
|
||||
throw new ISE(
|
||||
e,
|
||||
"Unable to create the success file [%s] at the location [%s]",
|
||||
DurableStorageUtils.SUCCESS_MARKER_FILENAME,
|
||||
DurableStorageUtils.getSuccessFilePath(
|
||||
task.getControllerTaskId(),
|
||||
stageDef.getStageNumber(),
|
||||
task().getWorkerNumber()
|
||||
)
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
kernelManipulationQueue.add(holder -> holder.getStageKernelMap()
|
||||
.get(stageDef.getId())
|
||||
.setResultsComplete(resultObject));
|
||||
|
|
|
@ -106,12 +106,11 @@ public class ControllerChatHandler implements ChatHandler
|
|||
* See {@link ControllerClient#postWorkerWarning} for the client-side code that calls this API.
|
||||
*/
|
||||
@POST
|
||||
@Path("/workerWarning/{taskId}")
|
||||
@Path("/workerWarning")
|
||||
@Produces(MediaType.APPLICATION_JSON)
|
||||
@Consumes(MediaType.APPLICATION_JSON)
|
||||
public Response httpPostWorkerWarning(
|
||||
final List<MSQErrorReport> errorReport,
|
||||
@PathParam("taskId") final String taskId,
|
||||
@Context final HttpServletRequest req
|
||||
)
|
||||
{
|
||||
|
|
|
@ -122,12 +122,9 @@ public class IndexerControllerClient implements ControllerClient
|
|||
}
|
||||
|
||||
@Override
|
||||
public void postWorkerWarning(String workerId, List<MSQErrorReport> MSQErrorReports) throws IOException
|
||||
public void postWorkerWarning(List<MSQErrorReport> MSQErrorReports) throws IOException
|
||||
{
|
||||
final String path = StringUtils.format(
|
||||
"/workerWarning/%s",
|
||||
StringUtils.urlEncode(workerId)
|
||||
);
|
||||
final String path = "/workerWarning";
|
||||
|
||||
doRequest(
|
||||
new RequestBuilder(HttpMethod.POST, path)
|
||||
|
|
|
@ -57,7 +57,7 @@ public class MSQWarningReportSimplePublisher implements MSQWarningReportPublishe
|
|||
final MSQErrorReport warningReport = MSQErrorReport.fromException(taskId, host, stageNumber, e);
|
||||
|
||||
try {
|
||||
controllerClient.postWorkerWarning(workerId, ImmutableList.of(warningReport));
|
||||
controllerClient.postWorkerWarning(ImmutableList.of(warningReport));
|
||||
}
|
||||
catch (IOException e2) {
|
||||
throw new RuntimeException(e2);
|
||||
|
|
|
@ -20,6 +20,7 @@
|
|||
package org.apache.druid.msq.shuffle;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.druid.frame.channel.ReadableFrameChannel;
|
||||
import org.apache.druid.frame.channel.ReadableInputStreamFrameChannel;
|
||||
import org.apache.druid.java.util.common.IOE;
|
||||
|
@ -27,36 +28,36 @@ import org.apache.druid.java.util.common.ISE;
|
|||
import org.apache.druid.java.util.common.RetryUtils;
|
||||
import org.apache.druid.java.util.common.concurrent.Execs;
|
||||
import org.apache.druid.java.util.common.io.Closer;
|
||||
import org.apache.druid.java.util.common.logger.Logger;
|
||||
import org.apache.druid.msq.indexing.InputChannelFactory;
|
||||
import org.apache.druid.msq.kernel.StageId;
|
||||
import org.apache.druid.storage.StorageConnector;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.util.List;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.Executors;
|
||||
import java.util.function.Supplier;
|
||||
|
||||
/**
|
||||
* Provides input channels connected to durable storage.
|
||||
*/
|
||||
public class DurableStorageInputChannelFactory implements InputChannelFactory
|
||||
{
|
||||
|
||||
private static final Logger LOG = new Logger(DurableStorageInputChannelFactory.class);
|
||||
|
||||
private final StorageConnector storageConnector;
|
||||
private final ExecutorService remoteInputStreamPool;
|
||||
private final String controllerTaskId;
|
||||
private final Supplier<List<String>> taskList;
|
||||
|
||||
public DurableStorageInputChannelFactory(
|
||||
final String controllerTaskId,
|
||||
final Supplier<List<String>> taskList,
|
||||
final StorageConnector storageConnector,
|
||||
final ExecutorService remoteInputStreamPool
|
||||
)
|
||||
{
|
||||
this.controllerTaskId = Preconditions.checkNotNull(controllerTaskId, "controllerTaskId");
|
||||
this.taskList = Preconditions.checkNotNull(taskList, "taskList");
|
||||
this.storageConnector = Preconditions.checkNotNull(storageConnector, "storageConnector");
|
||||
this.remoteInputStreamPool = Preconditions.checkNotNull(remoteInputStreamPool, "remoteInputStreamPool");
|
||||
}
|
||||
|
@ -67,7 +68,6 @@ public class DurableStorageInputChannelFactory implements InputChannelFactory
|
|||
*/
|
||||
public static DurableStorageInputChannelFactory createStandardImplementation(
|
||||
final String controllerTaskId,
|
||||
final Supplier<List<String>> taskList,
|
||||
final StorageConnector storageConnector,
|
||||
final Closer closer
|
||||
)
|
||||
|
@ -75,28 +75,35 @@ public class DurableStorageInputChannelFactory implements InputChannelFactory
|
|||
final ExecutorService remoteInputStreamPool =
|
||||
Executors.newCachedThreadPool(Execs.makeThreadFactory(controllerTaskId + "-remote-fetcher-%d"));
|
||||
closer.register(remoteInputStreamPool::shutdownNow);
|
||||
return new DurableStorageInputChannelFactory(controllerTaskId, taskList, storageConnector, remoteInputStreamPool);
|
||||
return new DurableStorageInputChannelFactory(controllerTaskId, storageConnector, remoteInputStreamPool);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReadableFrameChannel openChannel(StageId stageId, int workerNumber, int partitionNumber) throws IOException
|
||||
{
|
||||
final String workerTaskId = taskList.get().get(workerNumber);
|
||||
|
||||
try {
|
||||
final String remotePartitionPath = DurableStorageOutputChannelFactory.getPartitionFileName(
|
||||
final String remotePartitionPath = findSuccessfulPartitionOutput(
|
||||
controllerTaskId,
|
||||
workerTaskId,
|
||||
workerNumber,
|
||||
stageId.getStageNumber(),
|
||||
partitionNumber
|
||||
);
|
||||
LOG.debug(
|
||||
"Reading output of stage [%d], partition [%d] for worker [%d] from the file at path [%s]",
|
||||
stageId.getStageNumber(),
|
||||
partitionNumber,
|
||||
workerNumber,
|
||||
remotePartitionPath
|
||||
);
|
||||
RetryUtils.retry(() -> {
|
||||
if (!storageConnector.pathExists(remotePartitionPath)) {
|
||||
throw new ISE(
|
||||
"Could not find remote output of worker task[%s] stage[%d] partition[%d]",
|
||||
workerTaskId,
|
||||
"Could not find remote outputs of stage [%d] partition [%d] for worker [%d] at the path [%s]",
|
||||
stageId.getStageNumber(),
|
||||
partitionNumber
|
||||
partitionNumber,
|
||||
workerNumber,
|
||||
remotePartitionPath
|
||||
);
|
||||
}
|
||||
return Boolean.TRUE;
|
||||
|
@ -112,11 +119,64 @@ public class DurableStorageInputChannelFactory implements InputChannelFactory
|
|||
catch (Exception e) {
|
||||
throw new IOE(
|
||||
e,
|
||||
"Could not find remote output of worker task[%s] stage[%d] partition[%d]",
|
||||
workerTaskId,
|
||||
"Encountered error while reading the output of stage [%d], partition [%d] for worker [%d]",
|
||||
stageId.getStageNumber(),
|
||||
partitionNumber
|
||||
partitionNumber,
|
||||
workerNumber
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* 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 outputs of stage [%d], partition [%d] for the worker [%d]",
|
||||
successfulFilePath,
|
||||
stageNumber,
|
||||
partitionNumber,
|
||||
workerNo
|
||||
);
|
||||
}
|
||||
|
||||
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.debug(
|
||||
"Reading output of stage [%d], partition [%d] from task id [%s]",
|
||||
stageNumber,
|
||||
partitionNumber,
|
||||
successfulTaskId
|
||||
);
|
||||
|
||||
return DurableStorageUtils.getPartitionOutputsFileNameForPartition(
|
||||
controllerTaskId,
|
||||
stageNumber,
|
||||
workerNo,
|
||||
successfulTaskId,
|
||||
partitionNumber
|
||||
);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -20,7 +20,6 @@
|
|||
package org.apache.druid.msq.shuffle;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import org.apache.druid.common.utils.IdUtils;
|
||||
import org.apache.druid.frame.allocation.ArenaMemoryAllocator;
|
||||
import org.apache.druid.frame.channel.ReadableNilFrameChannel;
|
||||
import org.apache.druid.frame.channel.WritableFrameFileChannel;
|
||||
|
@ -28,31 +27,39 @@ import org.apache.druid.frame.file.FrameFileWriter;
|
|||
import org.apache.druid.frame.processor.OutputChannel;
|
||||
import org.apache.druid.frame.processor.OutputChannelFactory;
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.java.util.common.logger.Logger;
|
||||
import org.apache.druid.storage.StorageConnector;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.OutputStreamWriter;
|
||||
import java.nio.channels.Channels;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
|
||||
public class DurableStorageOutputChannelFactory implements OutputChannelFactory
|
||||
{
|
||||
|
||||
private static final Logger LOG = new Logger(DurableStorageOutputChannelFactory.class);
|
||||
|
||||
private final String controllerTaskId;
|
||||
private final String workerTaskId;
|
||||
private final int workerNumber;
|
||||
private final int stageNumber;
|
||||
private final String taskId;
|
||||
private final int frameSize;
|
||||
private final StorageConnector storageConnector;
|
||||
|
||||
public DurableStorageOutputChannelFactory(
|
||||
final String controllerTaskId,
|
||||
final String workerTaskId,
|
||||
final int workerNumber,
|
||||
final int stageNumber,
|
||||
final String taskId,
|
||||
final int frameSize,
|
||||
final StorageConnector storageConnector
|
||||
)
|
||||
{
|
||||
this.controllerTaskId = Preconditions.checkNotNull(controllerTaskId, "controllerTaskId");
|
||||
this.workerTaskId = Preconditions.checkNotNull(workerTaskId, "workerTaskId");
|
||||
this.workerNumber = workerNumber;
|
||||
this.stageNumber = stageNumber;
|
||||
this.taskId = taskId;
|
||||
this.frameSize = frameSize;
|
||||
this.storageConnector = Preconditions.checkNotNull(storageConnector, "storageConnector");
|
||||
}
|
||||
|
@ -63,16 +70,18 @@ public class DurableStorageOutputChannelFactory implements OutputChannelFactory
|
|||
*/
|
||||
public static DurableStorageOutputChannelFactory createStandardImplementation(
|
||||
final String controllerTaskId,
|
||||
final String workerTaskId,
|
||||
final int workerNumber,
|
||||
final int stageNumber,
|
||||
final String taskId,
|
||||
final int frameSize,
|
||||
final StorageConnector storageConnector
|
||||
)
|
||||
{
|
||||
return new DurableStorageOutputChannelFactory(
|
||||
controllerTaskId,
|
||||
workerTaskId,
|
||||
workerNumber,
|
||||
stageNumber,
|
||||
taskId,
|
||||
frameSize,
|
||||
storageConnector
|
||||
);
|
||||
|
@ -81,7 +90,13 @@ public class DurableStorageOutputChannelFactory implements OutputChannelFactory
|
|||
@Override
|
||||
public OutputChannel openChannel(int partitionNumber) throws IOException
|
||||
{
|
||||
final String fileName = getPartitionFileName(controllerTaskId, workerTaskId, stageNumber, partitionNumber);
|
||||
final String fileName = DurableStorageUtils.getPartitionOutputsFileNameForPartition(
|
||||
controllerTaskId,
|
||||
stageNumber,
|
||||
workerNumber,
|
||||
taskId,
|
||||
partitionNumber
|
||||
);
|
||||
final WritableFrameFileChannel writableChannel =
|
||||
new WritableFrameFileChannel(
|
||||
FrameFileWriter.open(
|
||||
|
@ -101,7 +116,13 @@ public class DurableStorageOutputChannelFactory implements OutputChannelFactory
|
|||
@Override
|
||||
public OutputChannel openNilChannel(int partitionNumber)
|
||||
{
|
||||
final String fileName = getPartitionFileName(controllerTaskId, workerTaskId, stageNumber, partitionNumber);
|
||||
final String fileName = DurableStorageUtils.getPartitionOutputsFileNameForPartition(
|
||||
controllerTaskId,
|
||||
stageNumber,
|
||||
workerNumber,
|
||||
taskId,
|
||||
partitionNumber
|
||||
);
|
||||
// As tasks dependent on output of this partition will forever block if no file is present in RemoteStorage. Hence, writing a dummy frame.
|
||||
try {
|
||||
|
||||
|
@ -111,32 +132,30 @@ public class DurableStorageOutputChannelFactory implements OutputChannelFactory
|
|||
catch (IOException e) {
|
||||
throw new ISE(
|
||||
e,
|
||||
"Unable to create empty remote output of workerTask[%s] stage[%d] partition[%d]",
|
||||
workerTaskId,
|
||||
"Unable to create empty remote output of stage [%d], partition [%d] for worker [%d]",
|
||||
stageNumber,
|
||||
partitionNumber
|
||||
partitionNumber,
|
||||
workerNumber
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
public static String getControllerDirectory(final String controllerTaskId)
|
||||
/**
|
||||
* 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.
|
||||
* If the success file is already present in the location, then this method is a noop
|
||||
*/
|
||||
public void createSuccessFile(String taskId) throws IOException
|
||||
{
|
||||
return StringUtils.format("controller_%s", IdUtils.validateId("controller task ID", controllerTaskId));
|
||||
String fileName = DurableStorageUtils.getSuccessFilePath(controllerTaskId, stageNumber, workerNumber);
|
||||
if (storageConnector.pathExists(fileName)) {
|
||||
LOG.warn("Path [%s] already exists. Won't attempt to rewrite on top of it.", fileName);
|
||||
return;
|
||||
}
|
||||
|
||||
public static String getPartitionFileName(
|
||||
final String controllerTaskId,
|
||||
final String workerTaskId,
|
||||
final int stageNumber,
|
||||
final int partitionNumber
|
||||
)
|
||||
{
|
||||
return StringUtils.format(
|
||||
"%s/worker_%s/stage_%d/part_%d",
|
||||
getControllerDirectory(controllerTaskId),
|
||||
IdUtils.validateId("worker task ID", workerTaskId),
|
||||
stageNumber,
|
||||
partitionNumber
|
||||
);
|
||||
OutputStreamWriter os = new OutputStreamWriter(storageConnector.write(fileName), StandardCharsets.UTF_8);
|
||||
os.write(taskId); // Add some dummy content in the file
|
||||
os.close();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,105 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.msq.shuffle;
|
||||
|
||||
import org.apache.druid.common.utils.IdUtils;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
|
||||
/**
|
||||
* Helper class that fetches the directory and file names corresponding to file location
|
||||
*/
|
||||
public class DurableStorageUtils
|
||||
{
|
||||
public static final String SUCCESS_MARKER_FILENAME = "__success";
|
||||
|
||||
public static String getControllerDirectory(final String controllerTaskId)
|
||||
{
|
||||
return StringUtils.format("controller_%s", IdUtils.validateId("controller task ID", controllerTaskId));
|
||||
}
|
||||
|
||||
public static String getSuccessFilePath(
|
||||
final String controllerTaskId,
|
||||
final int stageNumber,
|
||||
final int workerNumber
|
||||
)
|
||||
{
|
||||
String folderName = getWorkerOutputFolderName(
|
||||
controllerTaskId,
|
||||
stageNumber,
|
||||
workerNumber
|
||||
);
|
||||
String fileName = StringUtils.format("%s/%s", folderName, SUCCESS_MARKER_FILENAME);
|
||||
return fileName;
|
||||
}
|
||||
|
||||
/**
|
||||
* Fetches the directory location where workers will store the partition files corresponding to the stage number
|
||||
*/
|
||||
public static String getWorkerOutputFolderName(
|
||||
final String controllerTaskId,
|
||||
final int stageNumber,
|
||||
final int workerNumber
|
||||
)
|
||||
{
|
||||
return StringUtils.format(
|
||||
"%s/stage_%d/worker_%d",
|
||||
getControllerDirectory(controllerTaskId),
|
||||
stageNumber,
|
||||
workerNumber
|
||||
);
|
||||
}
|
||||
|
||||
/**
|
||||
* Fetches the directory location where a particular worker will store the partition files corresponding to the
|
||||
* stage number, and it's task id
|
||||
*/
|
||||
public static String getTaskIdOutputsFolderName(
|
||||
final String controllerTaskId,
|
||||
final int stageNumber,
|
||||
final int workerNumber,
|
||||
final String taskId
|
||||
)
|
||||
{
|
||||
return StringUtils.format(
|
||||
"%s/taskId_%s",
|
||||
getWorkerOutputFolderName(controllerTaskId, stageNumber, workerNumber),
|
||||
taskId
|
||||
);
|
||||
}
|
||||
|
||||
/**
|
||||
* Fetches the file location where a particular worker writes the data corresponding to a particular stage
|
||||
* and partition
|
||||
*/
|
||||
public static String getPartitionOutputsFileNameForPartition(
|
||||
final String controllerTaskId,
|
||||
final int stageNumber,
|
||||
final int workerNumber,
|
||||
final String taskId,
|
||||
final int partitionNumber
|
||||
)
|
||||
{
|
||||
return StringUtils.format(
|
||||
"%s/part_%d",
|
||||
getTaskIdOutputsFolderName(controllerTaskId, stageNumber, workerNumber, taskId),
|
||||
partitionNumber
|
||||
);
|
||||
}
|
||||
}
|
|
@ -31,6 +31,7 @@ import org.apache.druid.msq.indexing.ColumnMapping;
|
|||
import org.apache.druid.msq.indexing.ColumnMappings;
|
||||
import org.apache.druid.msq.indexing.MSQSpec;
|
||||
import org.apache.druid.msq.indexing.MSQTuningConfig;
|
||||
import org.apache.druid.msq.shuffle.DurableStorageUtils;
|
||||
import org.apache.druid.msq.test.MSQTestBase;
|
||||
import org.apache.druid.query.InlineDataSource;
|
||||
import org.apache.druid.query.QueryDataSource;
|
||||
|
@ -62,6 +63,8 @@ import org.apache.druid.sql.calcite.util.CalciteTests;
|
|||
import org.hamcrest.CoreMatchers;
|
||||
import org.junit.Test;
|
||||
import org.junit.internal.matchers.ThrowableMessageMatcher;
|
||||
import org.mockito.ArgumentMatchers;
|
||||
import org.mockito.Mockito;
|
||||
|
||||
import javax.annotation.Nonnull;
|
||||
import java.io.File;
|
||||
|
@ -1203,6 +1206,54 @@ public class MSQSelectTest extends MSQTestBase
|
|||
.verifyResults();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGroupByOnFooWithDurableStoragePathAssertions() throws IOException
|
||||
{
|
||||
RowSignature rowSignature = RowSignature.builder()
|
||||
.add("cnt", ColumnType.LONG)
|
||||
.add("cnt1", ColumnType.LONG)
|
||||
.build();
|
||||
|
||||
testSelectQuery()
|
||||
.setSql("select cnt,count(*) as cnt1 from foo group by cnt")
|
||||
.setExpectedMSQSpec(MSQSpec.builder()
|
||||
.query(GroupByQuery.builder()
|
||||
.setDataSource(CalciteTests.DATASOURCE1)
|
||||
.setInterval(querySegmentSpec(Filtration
|
||||
.eternity()))
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setDimensions(dimensions(
|
||||
new DefaultDimensionSpec(
|
||||
"cnt",
|
||||
"d0",
|
||||
ColumnType.LONG
|
||||
)
|
||||
))
|
||||
.setAggregatorSpecs(aggregators(new CountAggregatorFactory(
|
||||
"a0")))
|
||||
.setContext(DEFAULT_MSQ_CONTEXT)
|
||||
.build())
|
||||
.columnMappings(
|
||||
new ColumnMappings(ImmutableList.of(
|
||||
new ColumnMapping("d0", "cnt"),
|
||||
new ColumnMapping("a0", "cnt1")
|
||||
)
|
||||
))
|
||||
.tuningConfig(MSQTuningConfig.defaultConfig())
|
||||
.build())
|
||||
.setExpectedRowSignature(rowSignature)
|
||||
.setExpectedResultRows(ImmutableList.of(new Object[]{1L, 6L}))
|
||||
.verifyResults();
|
||||
File successFile = new File(
|
||||
localFileStorageDir,
|
||||
DurableStorageUtils.getSuccessFilePath("query-test-query", 0, 0)
|
||||
);
|
||||
|
||||
Mockito.verify(localFileStorageConnector, Mockito.times(2))
|
||||
.write(ArgumentMatchers.endsWith("__success"));
|
||||
}
|
||||
|
||||
|
||||
@Nonnull
|
||||
private List<Object[]> expectedMultiValueFooRowsGroup()
|
||||
{
|
||||
|
|
|
@ -43,7 +43,6 @@ import org.apache.druid.segment.TestHelper;
|
|||
import org.apache.druid.segment.column.ColumnType;
|
||||
import org.apache.druid.segment.column.RowSignature;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Ignore;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.rules.TemporaryFolder;
|
||||
|
@ -175,7 +174,6 @@ public class MSQTaskReportTest
|
|||
}
|
||||
|
||||
@Test
|
||||
@Ignore("requires https://github.com/apache/druid/pull/12938")
|
||||
public void testWriteTaskReport() throws Exception
|
||||
{
|
||||
final MSQTaskReport report = new MSQTaskReport(
|
||||
|
@ -204,7 +202,9 @@ public class MSQTaskReportTest
|
|||
|
||||
final Map<String, TaskReport> reportMap = mapper.readValue(
|
||||
reportFile,
|
||||
new TypeReference<Map<String, TaskReport>>() {}
|
||||
new TypeReference<Map<String, TaskReport>>()
|
||||
{
|
||||
}
|
||||
);
|
||||
|
||||
final MSQTaskReport report2 = (MSQTaskReport) reportMap.get(MSQTaskReport.REPORT_KEY);
|
||||
|
|
|
@ -140,7 +140,7 @@ import org.apache.druid.sql.calcite.util.SqlTestFramework;
|
|||
import org.apache.druid.sql.calcite.view.InProcessViewManager;
|
||||
import org.apache.druid.storage.StorageConnector;
|
||||
import org.apache.druid.storage.StorageConnectorProvider;
|
||||
import org.apache.druid.storage.local.LocalFileStorageConnectorProvider;
|
||||
import org.apache.druid.storage.local.LocalFileStorageConnector;
|
||||
import org.apache.druid.timeline.DataSegment;
|
||||
import org.apache.druid.timeline.PruneLoadSpec;
|
||||
import org.apache.druid.timeline.SegmentId;
|
||||
|
@ -218,6 +218,7 @@ public class MSQTestBase extends BaseCalciteQueryTest
|
|||
public final boolean useDefault = NullHandling.replaceWithDefault();
|
||||
|
||||
protected File localFileStorageDir;
|
||||
protected LocalFileStorageConnector localFileStorageConnector;
|
||||
private static final Logger log = new Logger(MSQTestBase.class);
|
||||
private ObjectMapper objectMapper;
|
||||
private MSQTestOverlordServiceClient indexingServiceClient;
|
||||
|
@ -327,8 +328,11 @@ public class MSQTestBase extends BaseCalciteQueryTest
|
|||
MultiStageQuery.class
|
||||
);
|
||||
localFileStorageDir = tmpFolder.newFolder("fault");
|
||||
localFileStorageConnector = Mockito.spy(
|
||||
new LocalFileStorageConnector(localFileStorageDir)
|
||||
);
|
||||
binder.bind(Key.get(StorageConnector.class, MultiStageQuery.class))
|
||||
.toProvider(new LocalFileStorageConnectorProvider(localFileStorageDir));
|
||||
.toProvider(() -> localFileStorageConnector);
|
||||
}
|
||||
catch (IOException e) {
|
||||
throw new ISE(e, "Unable to create setup storage connector");
|
||||
|
|
|
@ -75,7 +75,7 @@ public class MSQTestControllerClient implements ControllerClient
|
|||
}
|
||||
|
||||
@Override
|
||||
public void postWorkerWarning(String workerId, List<MSQErrorReport> MSQErrorReports)
|
||||
public void postWorkerWarning(List<MSQErrorReport> MSQErrorReports)
|
||||
{
|
||||
controller.workerWarning(MSQErrorReports);
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue