From 3a24539eb21c6640e6d6debc29f610d25fdc267c Mon Sep 17 00:00:00 2001 From: Rohan Garg Date: Mon, 14 Nov 2022 15:56:30 +0530 Subject: [PATCH 01/15] Allow using durable storage for SuperSorter intermediate data --- .../druid/storage/StorageConnector.java | 15 +- .../local/LocalFileStorageConnector.java | 15 +- .../local/LocalFileStorageConnectorTest.java | 28 +++ .../apache/druid/msq/exec/ControllerImpl.java | 8 +- .../org/apache/druid/msq/exec/WorkerImpl.java | 60 ++++-- .../CountingOutputChannelFactory.java | 18 +- .../CountingWritableFrameChannel.java | 7 +- .../msq/indexing/DurableStorageCleaner.java | 2 +- .../druid/msq/indexing/MSQControllerTask.java | 2 +- .../DurableStorageInputChannelFactory.java | 4 +- .../shuffle/WorkerInputChannelFactory.java | 2 +- .../msq/util/MultiStageQueryContext.java | 22 ++- .../apache/druid/msq/exec/MSQSelectTest.java | 2 +- .../indexing/DurableStorageCleanerTest.java | 2 +- .../msq/util/MultiStageQueryContextTest.java | 39 +++- .../storage/s3/output/S3StorageConnector.java | 29 +++ .../s3/output/S3StorageConnectorTest.java | 51 +++++ .../DurableStorageOutputChannelFactory.java | 137 +++++++++++++- ...toragePartitionedReadableFrameChannel.java | 89 +++++++++ .../frame/channel}/DurableStorageUtils.java | 18 +- .../PartitionedReadableFrameChannel.java | 36 ++++ .../ReadableByteChunksFrameChannel.java | 15 +- .../ReadableInputStreamFrameChannel.java | 5 +- .../apache/druid/frame/file/FrameFile.java | 85 ++------- .../druid/frame/file/FrameFileFooter.java | 126 +++++++++++++ .../BlockingQueueOutputChannelFactory.java | 8 + .../processor/FileOutputChannelFactory.java | 58 ++++++ .../frame/processor/OutputChannelFactory.java | 5 + .../processor/PartitionedOutputChannel.java | 117 ++++++++++++ .../druid/frame/processor/SuperSorter.java | 176 ++++++++---------- .../ReadableByteChunksFrameChannelTest.java | 14 +- .../ReadableInputStreamFrameChannelTest.java | 21 ++- .../FrameFileHttpResponseHandlerTest.java | 2 +- .../frame/processor/SuperSorterTest.java | 98 ++++++++-- 34 files changed, 1060 insertions(+), 256 deletions(-) rename {extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle => processing/src/main/java/org/apache/druid/frame/channel}/DurableStorageOutputChannelFactory.java (51%) create mode 100644 processing/src/main/java/org/apache/druid/frame/channel/DurableStoragePartitionedReadableFrameChannel.java rename {extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle => processing/src/main/java/org/apache/druid/frame/channel}/DurableStorageUtils.java (88%) create mode 100644 processing/src/main/java/org/apache/druid/frame/channel/PartitionedReadableFrameChannel.java create mode 100644 processing/src/main/java/org/apache/druid/frame/file/FrameFileFooter.java create mode 100644 processing/src/main/java/org/apache/druid/frame/processor/PartitionedOutputChannel.java diff --git a/core/src/main/java/org/apache/druid/storage/StorageConnector.java b/core/src/main/java/org/apache/druid/storage/StorageConnector.java index adafac1a1ad8..e643882e4915 100644 --- a/core/src/main/java/org/apache/druid/storage/StorageConnector.java +++ b/core/src/main/java/org/apache/druid/storage/StorageConnector.java @@ -65,7 +65,7 @@ public interface StorageConnector boolean pathExists(String path) throws IOException; /** - * Reads the data present at the path the underlying storage system. Most implementations prepend the input path + * Reads the data present at the path in the underlying storage system. Most implementations prepend the input path * with a basePath. * The caller should take care of closing the stream when done or in case of error. * @@ -75,6 +75,19 @@ public interface StorageConnector */ InputStream read(String path) throws IOException; + /** + * Reads the data present for a given range at the path in the underlying storage system. + * Most implementations prepend the input path with a basePath. + * The caller should take care of closing the stream when done or in case of error. Further, the caller must ensure + * that the start offset and the size of the read are valid parameters for the given path for correct behavior. + * @param path The path to read data from + * @param from Start offset of the read in the path + * @param size Length of the read to be done + * @return InputStream starting from the given offset limited by the given size + * @throws IOException if the path is not present or the unable to read the data present on the path + */ + InputStream readRange(String path, long from, long size) throws IOException; + /** * Open an {@link OutputStream} for writing data to the path in the underlying storage system. * Most implementations prepend the input path with a basePath. diff --git a/core/src/main/java/org/apache/druid/storage/local/LocalFileStorageConnector.java b/core/src/main/java/org/apache/druid/storage/local/LocalFileStorageConnector.java index 393d0ace768d..4456eced9039 100644 --- a/core/src/main/java/org/apache/druid/storage/local/LocalFileStorageConnector.java +++ b/core/src/main/java/org/apache/druid/storage/local/LocalFileStorageConnector.java @@ -19,6 +19,7 @@ package org.apache.druid.storage.local; +import org.apache.commons.io.input.BoundedInputStream; import org.apache.druid.java.util.common.FileUtils; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; @@ -29,7 +30,10 @@ import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; +import java.nio.channels.Channels; +import java.nio.channels.FileChannel; import java.nio.file.Files; +import java.nio.file.StandardOpenOption; import java.util.Arrays; import java.util.List; import java.util.stream.Collectors; @@ -55,16 +59,19 @@ public boolean pathExists(String path) return fileWithBasePath(path).exists(); } - /** - * 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. - */ @Override public InputStream read(String path) throws IOException { return Files.newInputStream(fileWithBasePath(path).toPath()); } + @Override + public InputStream readRange(String path, long from, long size) throws IOException + { + FileChannel fileChannel = FileChannel.open(fileWithBasePath(path).toPath(), StandardOpenOption.READ); + return new BoundedInputStream(Channels.newInputStream(fileChannel.position(from)), size); + } + /** * Writes the file present with the materialized location as basePath + path. * In case the parent directory does not exist, we create the parent dir recursively. diff --git a/core/src/test/java/org/apache/druid/storage/local/LocalFileStorageConnectorTest.java b/core/src/test/java/org/apache/druid/storage/local/LocalFileStorageConnectorTest.java index 610506129ec4..4fd24c672603 100644 --- a/core/src/test/java/org/apache/druid/storage/local/LocalFileStorageConnectorTest.java +++ b/core/src/test/java/org/apache/druid/storage/local/LocalFileStorageConnectorTest.java @@ -35,6 +35,7 @@ import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; +import java.nio.charset.StandardCharsets; import java.util.HashSet; import java.util.List; import java.util.Set; @@ -151,6 +152,33 @@ public void listFilesTest() throws Exception ); } + @Test + public void testReadRange() throws Exception + { + String uuid = UUID.randomUUID().toString(); + String data = "Hello"; + try (OutputStream outputStream = storageConnector.write(uuid)) { + outputStream.write(data.getBytes(StandardCharsets.UTF_8)); + } + + // normal reads + for (int start = 0; start < data.length(); start++) { + for (int length = 1; length <= data.length() - start; length++) { + InputStream is = storageConnector.readRange(uuid, start, length); + byte[] dataBytes = new byte[length]; + Assert.assertEquals(is.read(dataBytes), length); + Assert.assertEquals(is.read(), -1); // reading further produces no data + Assert.assertEquals(data.substring(start, start + length), new String(dataBytes, StandardCharsets.UTF_8)); + } + } + + // 0 size read + InputStream is = storageConnector.readRange(uuid, 0, 0); + byte[] dataBytes = new byte[0]; + Assert.assertEquals(is.read(dataBytes), -1); + Assert.assertEquals(data.substring(0, 0), new String(dataBytes, StandardCharsets.UTF_8)); + } + private void checkContents(String uuid) throws IOException { try (InputStream inputStream = storageConnector.read(uuid)) { diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java index a4812292fb15..0588ffe9dc54 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java @@ -40,6 +40,7 @@ import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.frame.allocation.ArenaMemoryAllocator; +import org.apache.druid.frame.channel.DurableStorageUtils; import org.apache.druid.frame.channel.FrameChannelSequence; import org.apache.druid.frame.key.ClusterBy; import org.apache.druid.frame.key.ClusterByPartition; @@ -147,7 +148,6 @@ 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.DurableStorageUtils; import org.apache.druid.msq.shuffle.WorkerInputChannelFactory; import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot; import org.apache.druid.msq.util.DimensionSchemaUtils; @@ -522,7 +522,7 @@ private QueryDefinition initializeQueryDefAndState(final Closer closer) closer.register(netClient::close); final boolean isDurableStorageEnabled = - MultiStageQueryContext.isDurableStorageEnabled(task.getQuerySpec().getQuery().context()); + MultiStageQueryContext.isDurableShuffleStorageEnabled(task.getQuerySpec().getQuery().context()); final QueryDefinition queryDef = makeQueryDefinition( id(), @@ -1187,7 +1187,7 @@ private Yielder getFinalResultsYielder( final InputChannelFactory inputChannelFactory; - if (MultiStageQueryContext.isDurableStorageEnabled(task.getQuerySpec().getQuery().context())) { + if (MultiStageQueryContext.isDurableShuffleStorageEnabled(task.getQuerySpec().getQuery().context())) { inputChannelFactory = DurableStorageInputChannelFactory.createStandardImplementation( id(), MSQTasks.makeStorageConnector(context.injector()), @@ -1289,7 +1289,7 @@ private void publishSegmentsIfNeeded( */ private void cleanUpDurableStorageIfNeeded() { - if (MultiStageQueryContext.isDurableStorageEnabled(task.getQuerySpec().getQuery().context())) { + if (MultiStageQueryContext.isDurableShuffleStorageEnabled(task.getQuerySpec().getQuery().context())) { final String controllerDirName = DurableStorageUtils.getControllerDirectory(task.getId()); try { // Delete all temporary files as a failsafe diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java index 5b68041d0e97..9206501ac720 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java @@ -32,6 +32,8 @@ import it.unimi.dsi.fastutil.bytes.ByteArrays; import org.apache.druid.frame.allocation.ArenaMemoryAllocator; import org.apache.druid.frame.channel.BlockingQueueFrameChannel; +import org.apache.druid.frame.channel.DurableStorageOutputChannelFactory; +import org.apache.druid.frame.channel.DurableStorageUtils; import org.apache.druid.frame.channel.ReadableFileFrameChannel; import org.apache.druid.frame.channel.ReadableFrameChannel; import org.apache.druid.frame.channel.ReadableNilFrameChannel; @@ -101,8 +103,6 @@ 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; @@ -160,6 +160,7 @@ public class WorkerImpl implements Worker private final ConcurrentHashMap> stageOutputs = new ConcurrentHashMap<>(); private final ConcurrentHashMap stageCounters = new ConcurrentHashMap<>(); private final boolean durableStageStorageEnabled; + private final boolean durableTaskIntermediateStorageEnabled; /** * Set once in {@link #runTask} and never reassigned. @@ -183,7 +184,10 @@ public WorkerImpl(MSQWorkerTask task, WorkerContext context) this.context = context; this.selfDruidNode = context.selfNode(); this.processorBouncer = context.processorBouncer(); - this.durableStageStorageEnabled = MultiStageQueryContext.isDurableStorageEnabled( + this.durableStageStorageEnabled = MultiStageQueryContext.isDurableShuffleStorageEnabled( + QueryContext.of(task.getContext()) + ); + this.durableTaskIntermediateStorageEnabled = MultiStageQueryContext.isDurableTaskIntermediateStorageEnabled( QueryContext.of(task.getContext()) ); } @@ -611,7 +615,8 @@ private OutputChannelFactory makeStageOutputChannelFactory(final FrameContext fr stageNumber, task().getId(), frameSize, - MSQTasks.makeStorageConnector(context.injector()) + MSQTasks.makeStorageConnector(context.injector()), + context.tempDir() ); } else { final File fileChannelDirectory = @@ -621,6 +626,31 @@ private OutputChannelFactory makeStageOutputChannelFactory(final FrameContext fr } } + private OutputChannelFactory makeSuperSorterIntermediateOutputChannelFactory( + final FrameContext frameContext, + final int stageNumber, + final File tmpDir + ) + { + final int frameSize = frameContext.memoryParameters().getLargeFrameSize(); + + if (durableTaskIntermediateStorageEnabled) { + return DurableStorageOutputChannelFactory.createStandardImplementation( + task.getControllerTaskId(), + task().getWorkerNumber(), + stageNumber, + task().getId(), + frameSize, + MSQTasks.makeStorageConnector(context.injector()), + tmpDir + ); + } else { + final File fileChannelDirectory = + new File(tmpDir, StringUtils.format("intermediate_output_stage_%06d", stageNumber)); + return new FileOutputChannelFactory(fileChannelDirectory, frameSize); + } + } + /** * Decorates the server-wide {@link QueryProcessingPool} such that any Callables and Runnables, not just * {@link PrioritizedCallable} and {@link PrioritizedRunnable}, may be added to it. @@ -821,16 +851,22 @@ private SettableFuture startWorkOrder( stagePartitionBoundariesFuture = Futures.immediateFuture(kernel.getResultPartitionBoundaries()); } + final File sorterTmpDir = new File(context.tempDir(), "super-sort-" + UUID.randomUUID()); + FileUtils.mkdirp(sorterTmpDir); + if (!sorterTmpDir.isDirectory()) { + throw new IOException("Cannot create directory: " + sorterTmpDir); + } + outputChannelsFuture = superSortOutputChannels( workOrder.getStageDefinition(), clusterBy, workerResultAndOutputChannels.getOutputChannels(), stagePartitionBoundariesFuture, shuffleOutputChannelFactory, + makeSuperSorterIntermediateOutputChannelFactory(frameContext, stageDef.getStageNumber(), sorterTmpDir), exec, cancellationId, frameContext.memoryParameters(), - context, kernelManipulationQueue, counters.sortProgress() ); @@ -908,7 +944,8 @@ public void onSuccess(final List workerResultAndOutputChannelsResolved) stageDef.getStageNumber(), task().getId(), frameContext.memoryParameters().getStandardFrameSize(), - MSQTasks.makeStorageConnector(context.injector()) + MSQTasks.makeStorageConnector(context.injector()), + context.tempDir() ); try { durableStorageOutputChannelFactory.createSuccessFile(task.getId()); @@ -1028,10 +1065,10 @@ private static ListenableFuture superSortOutputChannels( final OutputChannels processorOutputChannels, final ListenableFuture stagePartitionBoundariesFuture, final OutputChannelFactory outputChannelFactory, + final OutputChannelFactory intermediateOutputChannelFactory, final FrameProcessorExecutor exec, final String cancellationId, final WorkerMemoryParameters memoryParameters, - final WorkerContext context, final BlockingQueue> kernelManipulationQueue, final SuperSorterProgressTracker superSorterProgressTracker ) throws IOException @@ -1073,21 +1110,14 @@ private static ListenableFuture superSortOutputChannels( .collect(Collectors.toList()); } - final File sorterTmpDir = new File(context.tempDir(), "super-sort-" + UUID.randomUUID()); - FileUtils.mkdirp(sorterTmpDir); - if (!sorterTmpDir.isDirectory()) { - throw new IOException("Cannot create directory: " + sorterTmpDir); - } - final SuperSorter sorter = new SuperSorter( channelsToSuperSort, stageDefinition.getFrameReader(), clusterBy, stagePartitionBoundariesFuture, exec, - sorterTmpDir, outputChannelFactory, - () -> ArenaMemoryAllocator.createOnHeap(memoryParameters.getLargeFrameSize()), + intermediateOutputChannelFactory, memoryParameters.getSuperSorterMaxActiveProcessors(), memoryParameters.getSuperSorterMaxChannelsPerProcessor(), -1, diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/CountingOutputChannelFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/CountingOutputChannelFactory.java index 0521389d6670..9938948f9a0c 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/CountingOutputChannelFactory.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/CountingOutputChannelFactory.java @@ -22,6 +22,7 @@ import com.google.common.base.Preconditions; import org.apache.druid.frame.processor.OutputChannel; import org.apache.druid.frame.processor.OutputChannelFactory; +import org.apache.druid.frame.processor.PartitionedOutputChannel; import org.apache.druid.msq.counters.ChannelCounters; import java.io.IOException; @@ -49,8 +50,21 @@ public OutputChannel openChannel(int partitionNumber) throws IOException baseWritableChannel -> new CountingWritableFrameChannel( baseChannel.getWritableChannel(), - channelCounters, - baseChannel.getPartitionNumber() + channelCounters + ) + ); + } + + @Override + public PartitionedOutputChannel openChannel(String name, boolean deleteAfterRead) throws IOException + { + final PartitionedOutputChannel baseChannel = baseFactory.openChannel(name, deleteAfterRead); + + return baseChannel.mapWritableChannel( + baseWritableChannel -> + new CountingWritableFrameChannel( + baseChannel.getWritableChannel(), + channelCounters ) ); } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/CountingWritableFrameChannel.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/CountingWritableFrameChannel.java index 1daf6ef32ef7..1ee3ac69232d 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/CountingWritableFrameChannel.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/CountingWritableFrameChannel.java @@ -31,24 +31,21 @@ public class CountingWritableFrameChannel implements WritableFrameChannel { private final WritableFrameChannel baseChannel; private final ChannelCounters channelCounters; - private final int partitionNumber; public CountingWritableFrameChannel( final WritableFrameChannel baseChannel, - final ChannelCounters channelCounters, - final int partitionNumber + final ChannelCounters channelCounters ) { this.baseChannel = baseChannel; this.channelCounters = channelCounters; - this.partitionNumber = partitionNumber; } @Override public void write(FrameWithPartition frame) throws IOException { baseChannel.write(frame); - channelCounters.addFrame(partitionNumber, frame.frame()); + channelCounters.addFrame(frame.partition(), frame.frame()); } @Override diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/DurableStorageCleaner.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/DurableStorageCleaner.java index 822e3fd460b9..b08ec2de146c 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/DurableStorageCleaner.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/DurableStorageCleaner.java @@ -24,6 +24,7 @@ import com.google.common.collect.Sets; import com.google.inject.Inject; import com.google.inject.Provider; +import org.apache.druid.frame.channel.DurableStorageUtils; import org.apache.druid.indexing.overlord.TaskMaster; import org.apache.druid.indexing.overlord.TaskRunner; import org.apache.druid.indexing.overlord.TaskRunnerWorkItem; @@ -31,7 +32,6 @@ import org.apache.druid.java.util.common.concurrent.ScheduledExecutors; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.msq.guice.MultiStageQuery; -import org.apache.druid.msq.shuffle.DurableStorageUtils; import org.apache.druid.storage.StorageConnector; import org.joda.time.Duration; diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQControllerTask.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQControllerTask.java index 76d079348be3..bfb761a830b9 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQControllerTask.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQControllerTask.java @@ -106,7 +106,7 @@ public MSQControllerTask( this.sqlQueryContext = sqlQueryContext; this.sqlTypeNames = sqlTypeNames; - if (MultiStageQueryContext.isDurableStorageEnabled(querySpec.getQuery().context())) { + if (MultiStageQueryContext.isDurableShuffleStorageEnabled(querySpec.getQuery().context())) { this.remoteFetchExecutorService = Executors.newCachedThreadPool(Execs.makeThreadFactory(getId() + "-remote-fetcher-%d")); } else { diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/DurableStorageInputChannelFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/DurableStorageInputChannelFactory.java index 0d32a2d6bbbb..5645a00f5bab 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/DurableStorageInputChannelFactory.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/DurableStorageInputChannelFactory.java @@ -21,6 +21,7 @@ import com.google.common.base.Preconditions; import org.apache.commons.io.IOUtils; +import org.apache.druid.frame.channel.DurableStorageUtils; import org.apache.druid.frame.channel.ReadableFrameChannel; import org.apache.druid.frame.channel.ReadableInputStreamFrameChannel; import org.apache.druid.java.util.common.IOE; @@ -113,7 +114,8 @@ public ReadableFrameChannel openChannel(StageId stageId, int workerNumber, int p return ReadableInputStreamFrameChannel.open( inputStream, remotePartitionPath, - remoteInputStreamPool + remoteInputStreamPool, + false ); } catch (Exception e) { diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/WorkerInputChannelFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/WorkerInputChannelFactory.java index 4aa439106714..cfb7c64a1a7e 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/WorkerInputChannelFactory.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/WorkerInputChannelFactory.java @@ -52,7 +52,7 @@ public ReadableFrameChannel openChannel(StageId stageId, int workerNumber, int p { final String taskId = taskList.get().get(workerNumber); final ReadableByteChunksFrameChannel channel = - ReadableByteChunksFrameChannel.create(makeChannelId(taskId, stageId, partitionNumber)); + ReadableByteChunksFrameChannel.create(makeChannelId(taskId, stageId, partitionNumber), false); fetch(taskId, stageId, partitionNumber, 0, channel); return channel; } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MultiStageQueryContext.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MultiStageQueryContext.java index c7c61fd30c76..fc9dbb519f66 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MultiStageQueryContext.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MultiStageQueryContext.java @@ -25,6 +25,7 @@ import com.google.common.annotations.VisibleForTesting; import com.opencsv.RFC4180Parser; import com.opencsv.RFC4180ParserBuilder; +import org.apache.druid.java.util.common.IAE; import org.apache.druid.msq.kernel.WorkerAssignmentStrategy; import org.apache.druid.msq.sql.MSQMode; import org.apache.druid.query.QueryContext; @@ -60,6 +61,9 @@ public class MultiStageQueryContext public static final String CTX_ENABLE_DURABLE_SHUFFLE_STORAGE = "durableShuffleStorage"; private static final boolean DEFAULT_ENABLE_DURABLE_SHUFFLE_STORAGE = false; + public static final String CTX_ENABLE_DURABLE_TASK_INTERMEDIATE_STORAGE = "durableTaskIntermediateStorage"; + private static final boolean DEFAULT_ENABLE_DURABLE_TASK_INTERMEDIATE_STORAGE = false; + public static final String CTX_DESTINATION = "destination"; private static final String DEFAULT_DESTINATION = null; @@ -85,7 +89,7 @@ public static String getMSQMode(final QueryContext queryContext) ); } - public static boolean isDurableStorageEnabled(final QueryContext queryContext) + public static boolean isDurableShuffleStorageEnabled(final QueryContext queryContext) { return queryContext.getBoolean( CTX_ENABLE_DURABLE_SHUFFLE_STORAGE, @@ -93,6 +97,22 @@ public static boolean isDurableStorageEnabled(final QueryContext queryContext) ); } + public static boolean isDurableTaskIntermediateStorageEnabled(final QueryContext queryContext) + { + boolean intermediateStorageEnabled = queryContext.getBoolean( + CTX_ENABLE_DURABLE_TASK_INTERMEDIATE_STORAGE, + DEFAULT_ENABLE_DURABLE_TASK_INTERMEDIATE_STORAGE + ); + if (!isDurableShuffleStorageEnabled(queryContext) && intermediateStorageEnabled) { + throw new IAE( + "Cannot enable the feature %s without enabling %s", + CTX_ENABLE_DURABLE_TASK_INTERMEDIATE_STORAGE, + CTX_ENABLE_DURABLE_SHUFFLE_STORAGE + ); + } + return intermediateStorageEnabled; + } + public static boolean isFinalizeAggregations(final QueryContext queryContext) { return queryContext.getBoolean( diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java index c83481757d8f..717b6ae6b479 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java @@ -24,6 +24,7 @@ import org.apache.druid.common.config.NullHandling; import org.apache.druid.data.input.impl.JsonInputFormat; import org.apache.druid.data.input.impl.LocalInputSource; +import org.apache.druid.frame.channel.DurableStorageUtils; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.math.expr.ExprMacroTable; @@ -31,7 +32,6 @@ 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; diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/DurableStorageCleanerTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/DurableStorageCleanerTest.java index 2cb0c15d668e..948e91815250 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/DurableStorageCleanerTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/DurableStorageCleanerTest.java @@ -21,11 +21,11 @@ import com.google.common.base.Optional; import com.google.common.collect.ImmutableList; +import org.apache.druid.frame.channel.DurableStorageUtils; import org.apache.druid.indexer.TaskLocation; import org.apache.druid.indexing.overlord.TaskMaster; import org.apache.druid.indexing.overlord.TaskRunner; import org.apache.druid.indexing.overlord.TaskRunnerWorkItem; -import org.apache.druid.msq.shuffle.DurableStorageUtils; import org.apache.druid.storage.StorageConnector; import org.easymock.Capture; import org.easymock.EasyMock; diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/util/MultiStageQueryContextTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/util/MultiStageQueryContextTest.java index 0153cbc38ed3..27273b0251a3 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/util/MultiStageQueryContextTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/util/MultiStageQueryContextTest.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import org.apache.druid.java.util.common.IAE; import org.apache.druid.msq.kernel.WorkerAssignmentStrategy; import org.apache.druid.query.BadQueryContextException; import org.apache.druid.query.QueryContext; @@ -41,6 +42,7 @@ import static org.apache.druid.msq.util.MultiStageQueryContext.CTX_DESTINATION; import static org.apache.druid.msq.util.MultiStageQueryContext.CTX_ENABLE_DURABLE_SHUFFLE_STORAGE; +import static org.apache.druid.msq.util.MultiStageQueryContext.CTX_ENABLE_DURABLE_TASK_INTERMEDIATE_STORAGE; import static org.apache.druid.msq.util.MultiStageQueryContext.CTX_FINALIZE_AGGREGATIONS; import static org.apache.druid.msq.util.MultiStageQueryContext.CTX_MAX_NUM_TASKS; import static org.apache.druid.msq.util.MultiStageQueryContext.CTX_MSQ_MODE; @@ -53,16 +55,45 @@ public class MultiStageQueryContextTest { @Test - public void isDurableStorageEnabled_noParameterSetReturnsDefaultValue() + public void isDurableShuffleStorageEnabled_noParameterSetReturnsDefaultValue() { - Assert.assertFalse(MultiStageQueryContext.isDurableStorageEnabled(QueryContext.empty())); + Assert.assertFalse(MultiStageQueryContext.isDurableShuffleStorageEnabled(QueryContext.empty())); } @Test - public void isDurableStorageEnabled_parameterSetReturnsCorrectValue() + public void isDurableShuffleStorageEnabled_parameterSetReturnsCorrectValue() { Map propertyMap = ImmutableMap.of(CTX_ENABLE_DURABLE_SHUFFLE_STORAGE, "true"); - Assert.assertTrue(MultiStageQueryContext.isDurableStorageEnabled(QueryContext.of(propertyMap))); + Assert.assertTrue(MultiStageQueryContext.isDurableShuffleStorageEnabled(QueryContext.of(propertyMap))); + } + + @Test + public void isDurableTaskIntermediateStorageEnabled_noParameterSetReturnsDefaultValue() + { + Assert.assertFalse(MultiStageQueryContext.isDurableTaskIntermediateStorageEnabled(QueryContext.empty())); + } + + @Test + public void isDurableTaskIntermediateStorageEnabled_parameterSetReturnsCorrectValue() + { + Map propertyMap = ImmutableMap.of( + CTX_ENABLE_DURABLE_SHUFFLE_STORAGE, "true", + CTX_ENABLE_DURABLE_TASK_INTERMEDIATE_STORAGE, "true" + ); + Assert.assertTrue(MultiStageQueryContext.isDurableTaskIntermediateStorageEnabled(QueryContext.of(propertyMap))); + } + + @Test + public void isDurableIntermediateStorageEnabled_partialParameterSetThrows() + { + Map propertyMap = ImmutableMap.of( + CTX_ENABLE_DURABLE_SHUFFLE_STORAGE, "false", + CTX_ENABLE_DURABLE_TASK_INTERMEDIATE_STORAGE, "true" + ); + Assert.assertThrows( + IAE.class, + () -> MultiStageQueryContext.isDurableTaskIntermediateStorageEnabled(QueryContext.of(propertyMap)) + ); } @Test diff --git a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3StorageConnector.java b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3StorageConnector.java index 01822b24a465..aaeec35bc5a3 100644 --- a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3StorageConnector.java +++ b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3StorageConnector.java @@ -89,6 +89,35 @@ public InputStream open(GetObjectRequest object, long offset) ); } + @Override + public InputStream readRange(String path, long from, long size) throws IOException + { + return new RetryingInputStream<>( + new GetObjectRequest(config.getBucket(), objectPath(path)).withRange(from, from + size - 1), + new ObjectOpenFunction() + { + @Override + public InputStream open(GetObjectRequest object) + { + return s3Client.getObject(object).getObjectContent(); + } + + @Override + public InputStream open(GetObjectRequest object, long offset) + { + final GetObjectRequest offsetObjectRequest = new GetObjectRequest( + object.getBucketName(), + object.getKey() + ); + offsetObjectRequest.setRange(offset); + return open(offsetObjectRequest); + } + }, + S3Utils.S3RETRY, + config.getMaxRetry() + ); + } + @Override public OutputStream write(String path) throws IOException { diff --git a/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/S3StorageConnectorTest.java b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/S3StorageConnectorTest.java index 1eb0df57f980..a7a5b6095eef 100644 --- a/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/S3StorageConnectorTest.java +++ b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/S3StorageConnectorTest.java @@ -41,6 +41,7 @@ import java.io.BufferedReader; import java.io.ByteArrayInputStream; import java.io.IOException; +import java.io.InputStream; import java.io.InputStreamReader; import java.nio.charset.StandardCharsets; import java.util.Collections; @@ -117,6 +118,56 @@ public void pathRead() throws IOException EasyMock.reset(S3_CLIENT); } + @Test + public void testReadRange() throws IOException + { + String data = "test"; + + // normal reads + for (int start = 0; start < data.length(); start++) { + for (int length = 1; length <= data.length() - start; length++) { + String dataQueried = data.substring(start, start + length); + EasyMock.reset(S3_CLIENT); + S3Object s3Object = new S3Object(); + s3Object.setObjectContent( + new ByteArrayInputStream(dataQueried.getBytes(StandardCharsets.UTF_8)) + ); + EasyMock.expect( + S3_CLIENT.getObject( + new GetObjectRequest(BUCKET, PREFIX + "/" + TEST_FILE).withRange(start, start + length - 1) + ) + ).andReturn(s3Object); + EasyMock.replay(S3_CLIENT); + + InputStream is = storageConnector.readRange(TEST_FILE, start, length); + byte[] dataBytes = new byte[length]; + Assert.assertEquals(is.read(dataBytes), length); + Assert.assertEquals(is.read(), -1); // reading further produces no data + Assert.assertEquals(dataQueried, new String(dataBytes, StandardCharsets.UTF_8)); + EasyMock.reset(S3_CLIENT); + } + } + + // 0 size read + EasyMock.reset(S3_CLIENT); + S3Object s3Object = new S3Object(); + s3Object.setObjectContent( + new ByteArrayInputStream("".getBytes(StandardCharsets.UTF_8)) + ); + EasyMock.expect( + S3_CLIENT.getObject( + new GetObjectRequest(BUCKET, PREFIX + "/" + TEST_FILE).withRange(0, -1) + ) + ).andReturn(s3Object); + EasyMock.replay(S3_CLIENT); + + InputStream is = storageConnector.readRange(TEST_FILE, 0, 0); + byte[] dataBytes = new byte[0]; + Assert.assertEquals(is.read(dataBytes), -1); + Assert.assertEquals("", new String(dataBytes, StandardCharsets.UTF_8)); + EasyMock.reset(S3_CLIENT); + } + @Test public void pathDelete() throws IOException { diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/DurableStorageOutputChannelFactory.java b/processing/src/main/java/org/apache/druid/frame/channel/DurableStorageOutputChannelFactory.java similarity index 51% rename from extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/DurableStorageOutputChannelFactory.java rename to processing/src/main/java/org/apache/druid/frame/channel/DurableStorageOutputChannelFactory.java index 3bc0f72c8924..dddf451c85c3 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/DurableStorageOutputChannelFactory.java +++ b/processing/src/main/java/org/apache/druid/frame/channel/DurableStorageOutputChannelFactory.java @@ -17,27 +17,45 @@ * under the License. */ -package org.apache.druid.msq.shuffle; +package org.apache.druid.frame.channel; import com.google.common.base.Preconditions; +import com.google.common.base.Suppliers; +import com.google.common.io.CountingOutputStream; +import org.apache.commons.io.IOUtils; +import org.apache.datasketches.memory.Memory; +import org.apache.druid.frame.Frame; import org.apache.druid.frame.allocation.ArenaMemoryAllocator; -import org.apache.druid.frame.channel.ReadableNilFrameChannel; -import org.apache.druid.frame.channel.WritableFrameFileChannel; +import org.apache.druid.frame.file.FrameFileFooter; 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.frame.processor.PartitionedOutputChannel; +import org.apache.druid.java.util.common.FileUtils; import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.MappedByteBufferHandler; +import org.apache.druid.java.util.common.RetryUtils; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.storage.StorageConnector; +import java.io.File; +import java.io.FileOutputStream; import java.io.IOException; +import java.io.InputStream; import java.io.OutputStreamWriter; +import java.io.UncheckedIOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; import java.nio.channels.Channels; import java.nio.charset.StandardCharsets; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.function.Supplier; public class DurableStorageOutputChannelFactory implements OutputChannelFactory { - private static final Logger LOG = new Logger(DurableStorageOutputChannelFactory.class); private final String controllerTaskId; @@ -46,6 +64,8 @@ public class DurableStorageOutputChannelFactory implements OutputChannelFactory private final String taskId; private final int frameSize; private final StorageConnector storageConnector; + private final File tmpDir; + private final ExecutorService remoteInputStreamPool; public DurableStorageOutputChannelFactory( final String controllerTaskId, @@ -53,7 +73,8 @@ public DurableStorageOutputChannelFactory( final int stageNumber, final String taskId, final int frameSize, - final StorageConnector storageConnector + final StorageConnector storageConnector, + final File tmpDir ) { this.controllerTaskId = Preconditions.checkNotNull(controllerTaskId, "controllerTaskId"); @@ -62,6 +83,9 @@ public DurableStorageOutputChannelFactory( this.taskId = taskId; this.frameSize = frameSize; this.storageConnector = Preconditions.checkNotNull(storageConnector, "storageConnector"); + this.tmpDir = Preconditions.checkNotNull(tmpDir, "tmpDir is null"); + this.remoteInputStreamPool = + Executors.newCachedThreadPool(Execs.makeThreadFactory("-remote-fetcher-%d")); } /** @@ -74,7 +98,8 @@ public static DurableStorageOutputChannelFactory createStandardImplementation( final int stageNumber, final String taskId, final int frameSize, - final StorageConnector storageConnector + final StorageConnector storageConnector, + final File tmpDir ) { return new DurableStorageOutputChannelFactory( @@ -83,7 +108,8 @@ public static DurableStorageOutputChannelFactory createStandardImplementation( stageNumber, taskId, frameSize, - storageConnector + storageConnector, + tmpDir ); } @@ -108,11 +134,106 @@ public OutputChannel openChannel(int partitionNumber) throws IOException return OutputChannel.pair( writableChannel, ArenaMemoryAllocator.createOnHeap(frameSize), - () -> ReadableNilFrameChannel.INSTANCE, // remote reads should happen via the DurableStorageInputChannelFactory + () -> { + try { + RetryUtils.retry(() -> { + if (!storageConnector.pathExists(fileName)) { + throw new ISE("File does not exist : %s", fileName); + } + return Boolean.TRUE; + }, (throwable) -> true, 10); + } + catch (Exception exception) { + throw new RuntimeException(exception); + } + try { + return ReadableInputStreamFrameChannel.open( + storageConnector.read(fileName), + fileName, + remoteInputStreamPool, + false + ); + } + catch (IOException e) { + throw new UncheckedIOException(StringUtils.format("Unable to read file : %s", fileName), e); + } + }, partitionNumber ); } + @Override + public PartitionedOutputChannel openChannel(String name, boolean deleteAfterRead) throws IOException + { + final String fileName = DurableStorageUtils.getOutputsFileNameForPath( + controllerTaskId, + stageNumber, + workerNumber, + taskId, + name + ); + final CountingOutputStream countingOutputStream = new CountingOutputStream(storageConnector.write(fileName)); + final WritableFrameFileChannel writableChannel = + new WritableFrameFileChannel( + FrameFileWriter.open( + Channels.newChannel(countingOutputStream), + ByteBuffer.allocate(Frame.compressionBufferSize(frameSize)) + ) + ); + + final Supplier channelSizeSupplier = countingOutputStream::getCount; + + final File footerFile = new File(tmpDir, fileName + "_footer"); + // build supplier for reader the footer of the underlying frame file + final Supplier frameFileFooterSupplier = Suppliers.memoize(() -> { + try { + // read trailer and find the footer size + byte[] trailerBytes = new byte[FrameFileWriter.TRAILER_LENGTH]; + long channelSize = channelSizeSupplier.get(); + try (InputStream reader = storageConnector.readRange( + fileName, + channelSize - FrameFileWriter.TRAILER_LENGTH, + FrameFileWriter.TRAILER_LENGTH + )) { + int bytesRead = reader.read(trailerBytes, 0, trailerBytes.length); + if (bytesRead != FrameFileWriter.TRAILER_LENGTH) { + throw new RuntimeException("Invalid frame file trailer for object : " + fileName); + } + } + + Memory trailer = Memory.wrap(trailerBytes); + int footerLength = trailer.getInt(Integer.BYTES * 2L); + + // read the footer into a file and map it to memory + try (FileOutputStream footerFileStream = new FileOutputStream(footerFile); + InputStream footerInputStream = + storageConnector.readRange(fileName, channelSize - footerLength, footerLength)) { + IOUtils.copy(footerInputStream, footerFileStream); + } + MappedByteBufferHandler mapHandle = FileUtils.map(footerFile); + Memory footerMemory = Memory.wrap(mapHandle.get(), ByteOrder.LITTLE_ENDIAN); + + // create a frame file footer from the mapper memory + return new FrameFileFooter(footerMemory, channelSize); + } + catch (IOException e) { + throw new UncheckedIOException(e); + } + })::get; + + return PartitionedOutputChannel.pair( + writableChannel, + ArenaMemoryAllocator.createOnHeap(frameSize), + () -> new DurableStoragePartitionedReadableFrameChannel( + storageConnector, + frameFileFooterSupplier, + fileName, + remoteInputStreamPool, + footerFile + ) + ); + } + @Override public OutputChannel openNilChannel(int partitionNumber) { diff --git a/processing/src/main/java/org/apache/druid/frame/channel/DurableStoragePartitionedReadableFrameChannel.java b/processing/src/main/java/org/apache/druid/frame/channel/DurableStoragePartitionedReadableFrameChannel.java new file mode 100644 index 000000000000..ae3f529c8d92 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/frame/channel/DurableStoragePartitionedReadableFrameChannel.java @@ -0,0 +1,89 @@ +/* + * 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.frame.channel; + +import org.apache.druid.frame.file.FrameFileFooter; +import org.apache.druid.frame.file.FrameFileWriter; +import org.apache.druid.storage.StorageConnector; + +import java.io.File; +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.concurrent.ExecutorService; +import java.util.function.Supplier; + +public class DurableStoragePartitionedReadableFrameChannel implements PartitionedReadableFrameChannel +{ + private final StorageConnector storageConnector; + private final Supplier frameFileFooterSupplier; + private final String frameFileFullPath; + private final ExecutorService remoteInputStreamPool; + private final File footerFile; + + public DurableStoragePartitionedReadableFrameChannel( + StorageConnector storageConnector, + Supplier frameFileFooterSupplier, + String frameFileFullPath, + ExecutorService remoteInputStreamPool, + File footerFile + ) + { + this.storageConnector = storageConnector; + this.frameFileFooterSupplier = frameFileFooterSupplier; + this.frameFileFullPath = frameFileFullPath; + this.remoteInputStreamPool = remoteInputStreamPool; + this.footerFile = footerFile; + } + + @Override + public ReadableFrameChannel openChannel(int partitionNumber) + { + FrameFileFooter frameFileFooter = frameFileFooterSupplier.get(); + // find the range to read for partition + int startFrame = frameFileFooter.getPartitionStartFrame(partitionNumber); + int endFrame = frameFileFooter.getPartitionStartFrame(partitionNumber + 1); + long startByte = startFrame == 0 ? FrameFileWriter.MAGIC.length : frameFileFooter.getFrameEndPosition(startFrame - 1); + long endByte = frameFileFooter.getFrameEndPosition(endFrame - 1); + + try { + return ReadableInputStreamFrameChannel.open( + storageConnector.readRange(frameFileFullPath, startByte, endByte - startByte), + frameFileFullPath, + remoteInputStreamPool, + true + ); + } + catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + @Override + public void close() throws IOException + { + try { + storageConnector.deleteFile(frameFileFullPath); + footerFile.delete(); + } + catch (IOException e) { + throw new UncheckedIOException(e); + } + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/DurableStorageUtils.java b/processing/src/main/java/org/apache/druid/frame/channel/DurableStorageUtils.java similarity index 88% rename from extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/DurableStorageUtils.java rename to processing/src/main/java/org/apache/druid/frame/channel/DurableStorageUtils.java index df3b86a5c7ea..374d4e6820f2 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/DurableStorageUtils.java +++ b/processing/src/main/java/org/apache/druid/frame/channel/DurableStorageUtils.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.msq.shuffle; +package org.apache.druid.frame.channel; import org.apache.druid.common.utils.IdUtils; import org.apache.druid.java.util.common.StringUtils; @@ -102,4 +102,20 @@ public static String getPartitionOutputsFileNameForPartition( partitionNumber ); } + + + public static String getOutputsFileNameForPath( + final String controllerTaskId, + final int stageNumber, + final int workerNumber, + final String taskId, + final String path + ) + { + return StringUtils.format( + "%s/%s", + getTaskIdOutputsFolderName(controllerTaskId, stageNumber, workerNumber, taskId), + path + ); + } } diff --git a/processing/src/main/java/org/apache/druid/frame/channel/PartitionedReadableFrameChannel.java b/processing/src/main/java/org/apache/druid/frame/channel/PartitionedReadableFrameChannel.java new file mode 100644 index 000000000000..267bb76c9fe4 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/frame/channel/PartitionedReadableFrameChannel.java @@ -0,0 +1,36 @@ +/* + * 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.frame.channel; + +import java.io.Closeable; + +/** + * Provides an interface to read a partitioned frame channel. The channel might have frames with multiple partitions + * in it. + */ +public interface PartitionedReadableFrameChannel extends Closeable +{ + /** + * Allows reading a particular partition in the channel + * @param partitionNumber the partition to read + * @return a ReadableFrameChannel for the partition queried + */ + ReadableFrameChannel openChannel(int partitionNumber); +} diff --git a/processing/src/main/java/org/apache/druid/frame/channel/ReadableByteChunksFrameChannel.java b/processing/src/main/java/org/apache/druid/frame/channel/ReadableByteChunksFrameChannel.java index 2d659c2b304d..a4a40d70a38c 100644 --- a/processing/src/main/java/org/apache/druid/frame/channel/ReadableByteChunksFrameChannel.java +++ b/processing/src/main/java/org/apache/druid/frame/channel/ReadableByteChunksFrameChannel.java @@ -100,23 +100,27 @@ private enum StreamPart private long nextCompressedFrameLength = UNKNOWN_LENGTH; @GuardedBy("lock") - private StreamPart streamPart = StreamPart.MAGIC; + private StreamPart streamPart; - private ReadableByteChunksFrameChannel(String id, long bytesLimit) + private final boolean framesOnly; + + private ReadableByteChunksFrameChannel(String id, long bytesLimit, boolean framesOnly) { this.id = Preconditions.checkNotNull(id, "id"); this.bytesLimit = bytesLimit; + this.streamPart = framesOnly ? StreamPart.FRAMES : StreamPart.MAGIC; + this.framesOnly = framesOnly; } /** * Create a channel that aims to limit its memory footprint to one frame. The channel exerts backpressure * from {@link #addChunk} immediately once a full frame has been buffered. */ - public static ReadableByteChunksFrameChannel create(final String id) + public static ReadableByteChunksFrameChannel create(final String id, boolean framesOnly) { // Set byte limit to 1, so backpressure will be exerted as soon as we have a full frame buffered. // (The bytesLimit is soft: it will be exceeded if needed to store a complete frame.) - return new ReadableByteChunksFrameChannel(id, 1); + return new ReadableByteChunksFrameChannel(id, 1, framesOnly); } /** @@ -214,6 +218,9 @@ public boolean isFinished() public boolean canRead() { synchronized (lock) { + if (framesOnly) { + return canReadError() || canReadFrame(); + } // The noMoreWrites check is here so read() can throw an error if the last few chunks are an incomplete frame. return canReadError() || canReadFrame() || (streamPart != StreamPart.FOOTER && noMoreWrites); } diff --git a/processing/src/main/java/org/apache/druid/frame/channel/ReadableInputStreamFrameChannel.java b/processing/src/main/java/org/apache/druid/frame/channel/ReadableInputStreamFrameChannel.java index f06302b49200..280589da1d67 100644 --- a/processing/src/main/java/org/apache/druid/frame/channel/ReadableInputStreamFrameChannel.java +++ b/processing/src/main/java/org/apache/druid/frame/channel/ReadableInputStreamFrameChannel.java @@ -92,12 +92,13 @@ private ReadableInputStreamFrameChannel( public static ReadableInputStreamFrameChannel open( InputStream inputStream, String id, - ExecutorService executorService + ExecutorService executorService, + boolean framesOnly ) { final ReadableInputStreamFrameChannel channel = new ReadableInputStreamFrameChannel( inputStream, - ReadableByteChunksFrameChannel.create(id), + ReadableByteChunksFrameChannel.create(id, framesOnly), executorService ); diff --git a/processing/src/main/java/org/apache/druid/frame/file/FrameFile.java b/processing/src/main/java/org/apache/druid/frame/file/FrameFile.java index 1aab21533658..1f7c83c1c6f3 100644 --- a/processing/src/main/java/org/apache/druid/frame/file/FrameFile.java +++ b/processing/src/main/java/org/apache/druid/frame/file/FrameFile.java @@ -84,10 +84,8 @@ public enum Flag private final File file; private final long fileLength; - private final Memory footerMemory; // Footer is everything from the final MARKER_NO_MORE_FRAMES to EOF. + private final FrameFileFooter frameFileFooter; // Footer is everything from the final MARKER_NO_MORE_FRAMES to EOF. private final int maxMmapSize; - private final int numFrames; - private final int numPartitions; private final ReferenceCountingCloseableObject referenceCounter; private final Closeable referenceReleaser; @@ -116,21 +114,17 @@ public enum Flag private FrameFile( final File file, final long fileLength, - final Memory footerMemory, + final FrameFileFooter frameFileFooter, @Nullable final Memory wholeFileMemory, final int maxMmapSize, - final int numFrames, - final int numPartitions, final ReferenceCountingCloseableObject referenceCounter, final Closeable referenceReleaser ) { this.file = file; this.fileLength = fileLength; - this.footerMemory = footerMemory; + this.frameFileFooter = frameFileFooter; this.maxMmapSize = maxMmapSize; - this.numFrames = numFrames; - this.numPartitions = numPartitions; this.referenceCounter = referenceCounter; this.referenceReleaser = referenceReleaser; @@ -198,11 +192,7 @@ static FrameFile open(final File file, final int maxMmapSize, final Flag... flag randomAccessFile.seek(fileLength - FrameFileWriter.TRAILER_LENGTH); randomAccessFile.readFully(buf, 0, FrameFileWriter.TRAILER_LENGTH); - final int numFrames = bufMemory.getInt(0); - final int numPartitions = bufMemory.getInt(Integer.BYTES); final int footerLength = bufMemory.getInt(Integer.BYTES * 2L); - final int expectedFooterChecksum = bufMemory.getInt(Integer.BYTES * 3L); - if (footerLength < 0) { throw new ISE("Negative-size footer. Corrupt or truncated file?"); } else if (footerLength > fileLength) { @@ -234,23 +224,7 @@ static FrameFile open(final File file, final int maxMmapSize, final Flag... flag footerMemory = Memory.wrap(footerMapHandle.get(), ByteOrder.LITTLE_ENDIAN); } - // Verify footer begins with MARKER_NO_MORE_FRAMES. - if (footerMemory.getByte(0) != FrameFileWriter.MARKER_NO_MORE_FRAMES) { - throw new IOE("File [%s] end marker not in expected location", file); - } - - // Verify footer checksum. - final int actualChecksum = - (int) footerMemory.xxHash64(0, footerMemory.getCapacity() - Integer.BYTES, FrameFileWriter.CHECKSUM_SEED); - - if (expectedFooterChecksum != actualChecksum) { - throw new ISE("Expected footer checksum did not match actual checksum. Corrupt or truncated file?"); - } - - // Verify footer length. - if (footerLength != FrameFileWriter.footerLength(numFrames, numPartitions)) { - throw new ISE("Expected footer length did not match actual footer length. Corrupt or truncated file?"); - } + final FrameFileFooter frameFileFooter = new FrameFileFooter(footerMemory, fileLength); // Set up closer, refcounter; return instance. final Closer fileCloser = Closer.create(); @@ -270,11 +244,9 @@ static FrameFile open(final File file, final int maxMmapSize, final Flag... flag return new FrameFile( file, fileLength, - footerMemory, + frameFileFooter, wholeFileMemory, maxMmapSize, - numFrames, - numPartitions, referenceCounter, referenceCounter ); @@ -295,7 +267,7 @@ static FrameFile open(final File file, final int maxMmapSize, final Flag... flag */ public int numFrames() { - return numFrames; + return frameFileFooter.getNumFrames(); } /** @@ -303,7 +275,7 @@ public int numFrames() */ public int numPartitions() { - return numPartitions; + return frameFileFooter.getNumPartitions(); } /** @@ -313,21 +285,7 @@ public int numPartitions() public int getPartitionStartFrame(final int partition) { checkOpen(); - - if (partition < 0) { - throw new IAE("Partition [%,d] out of bounds", partition); - } else if (partition >= numPartitions) { - // Frame might not have every partition, if some are empty. - return numFrames; - } else { - final long partitionStartFrameLocation = - footerMemory.getCapacity() - - FrameFileWriter.TRAILER_LENGTH - - (long) numFrames * Long.BYTES - - (long) (numPartitions - partition) * Integer.BYTES; - - return footerMemory.getInt(partitionStartFrameLocation); - } + return frameFileFooter.getPartitionStartFrame(partition); } /** @@ -337,17 +295,17 @@ public Frame frame(final int frameNumber) { checkOpen(); - if (frameNumber < 0 || frameNumber >= numFrames) { + if (frameNumber < 0 || frameNumber >= numFrames()) { throw new IAE("Frame [%,d] out of bounds", frameNumber); } - final long frameEnd = getFrameEndPosition(frameNumber); + final long frameEnd = frameFileFooter.getFrameEndPosition(frameNumber); final long frameStart; if (frameNumber == 0) { frameStart = FrameFileWriter.MAGIC.length + Byte.BYTES /* MARKER_FRAME */; } else { - frameStart = getFrameEndPosition(frameNumber - 1) + Byte.BYTES /* MARKER_FRAME */; + frameStart = frameFileFooter.getFrameEndPosition(frameNumber - 1) + Byte.BYTES /* MARKER_FRAME */; } if (buffer == null || frameStart < bufferOffset || frameEnd > bufferOffset + buffer.getCapacity()) { @@ -377,11 +335,9 @@ public FrameFile newReference() return new FrameFile( file, fileLength, - footerMemory, + frameFileFooter, bufferOffset == 0 && bufferCloser == null ? buffer : null, // If bufferCloser is null, buffer is shared maxMmapSize, - numFrames, - numPartitions, referenceCounter, releaser ); @@ -414,23 +370,6 @@ private void checkOpen() } } - private long getFrameEndPosition(final int frameNumber) - { - assert frameNumber >= 0 && frameNumber < numFrames; - - final long frameEndPointerPosition = - footerMemory.getCapacity() - FrameFileWriter.TRAILER_LENGTH - (long) (numFrames - frameNumber) * Long.BYTES; - - final long frameEndPosition = footerMemory.getLong(frameEndPointerPosition); - - // Bounds check: protect against possibly-corrupt data. - if (frameEndPosition < 0 || frameEndPosition > fileLength - footerMemory.getCapacity()) { - throw new ISE("Corrupt frame file: frame [%,d] location out of range", frameNumber); - } - - return frameEndPosition; - } - /** * Updates {@link #buffer}, {@link #bufferOffset}, and {@link #bufferCloser} to a new offset. Closes the old * buffer, if any. diff --git a/processing/src/main/java/org/apache/druid/frame/file/FrameFileFooter.java b/processing/src/main/java/org/apache/druid/frame/file/FrameFileFooter.java new file mode 100644 index 000000000000..e460c1e57084 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/frame/file/FrameFileFooter.java @@ -0,0 +1,126 @@ +/* + * 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.frame.file; + +import org.apache.datasketches.memory.Memory; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.IOE; +import org.apache.druid.java.util.common.ISE; + +import java.nio.ByteOrder; + +/** + * Encapsulation for Æ’rame file footer related operations. The footer must be wrapped in a memory object (the memory + * can be physical or mmaped). Some verifications are also done on the footer to see if it is not corrupted. + */ +public class FrameFileFooter +{ + private final Memory footerMemory; + private final int numFrames; + private final int numPartitions; + private final long frameFileLength; + + public FrameFileFooter(Memory footerMemory, long frameFileLength) throws IOE + { + this.footerMemory = footerMemory; + this.frameFileLength = frameFileLength; + + Memory trailer = footerMemory.region( + footerMemory.getCapacity() - FrameFileWriter.TRAILER_LENGTH, + FrameFileWriter.TRAILER_LENGTH, + ByteOrder.LITTLE_ENDIAN + ); + this.numFrames = trailer.getInt(0); + this.numPartitions = trailer.getInt(Integer.BYTES); + int length = trailer.getInt(Integer.BYTES * 2L); + int expectedFooterChecksum = trailer.getInt(Integer.BYTES * 3L); + // Verify footer begins with MARKER_NO_MORE_FRAMES. + if (footerMemory.getByte(0) != FrameFileWriter.MARKER_NO_MORE_FRAMES) { + throw new IOE("File [%s] end marker not in expected location", "file"); + } + + // Verify footer checksum. + final int actualChecksum = + (int) footerMemory.xxHash64(0, footerMemory.getCapacity() - Integer.BYTES, FrameFileWriter.CHECKSUM_SEED); + + if (expectedFooterChecksum != actualChecksum) { + throw new ISE("Expected footer checksum did not match actual checksum. Corrupt or truncated file?"); + } + + // Verify footer length. + if (length != FrameFileWriter.footerLength(numFrames, numPartitions)) { + throw new ISE("Expected footer length did not match actual footer length. Corrupt or truncated file?"); + } + } + + /** + * First frame of a given partition. Partitions beyond {@link #getNumPartitions()} are treated as empty: if provided, + * this method returns {@link #getNumFrames()}. + */ + public int getPartitionStartFrame(final int partition) + { + if (partition < 0) { + throw new IAE("Partition [%,d] out of bounds", partition); + } else if (partition >= numPartitions) { + // Frame might not have every partition, if some are empty. + return numFrames; + } else { + final long partitionStartFrameLocation = + footerMemory.getCapacity() + - FrameFileWriter.TRAILER_LENGTH + - (long) numFrames * Long.BYTES + - (long) (numPartitions - partition) * Integer.BYTES; + + return footerMemory.getInt(partitionStartFrameLocation); + } + } + + /** + * Get the last byte for the frame specified. The byte number is offsetted from the frame file start and is exclusive. + * @param frameNumber the id of the frame to get the end position for + * @return a long exclusive index representing the frame end + */ + public long getFrameEndPosition(final int frameNumber) + { + assert frameNumber >= 0 && frameNumber < numFrames; + + final long frameEndPointerPosition = + footerMemory.getCapacity() - FrameFileWriter.TRAILER_LENGTH - (long) (numFrames - frameNumber) * Long.BYTES; + + final long frameEndPosition = footerMemory.getLong(frameEndPointerPosition); + + // Bounds check: protect against possibly-corrupt data. + if (frameEndPosition < 0 || frameEndPosition > frameFileLength - footerMemory.getCapacity()) { + throw new ISE("Corrupt frame file: frame [%,d] location out of range", frameNumber); + } + + return frameEndPosition; + } + + public int getNumFrames() + { + return numFrames; + } + + public int getNumPartitions() + { + return numPartitions; + } +} diff --git a/processing/src/main/java/org/apache/druid/frame/processor/BlockingQueueOutputChannelFactory.java b/processing/src/main/java/org/apache/druid/frame/processor/BlockingQueueOutputChannelFactory.java index 94c24b3220da..4bd3005307da 100644 --- a/processing/src/main/java/org/apache/druid/frame/processor/BlockingQueueOutputChannelFactory.java +++ b/processing/src/main/java/org/apache/druid/frame/processor/BlockingQueueOutputChannelFactory.java @@ -22,6 +22,8 @@ import org.apache.druid.frame.allocation.ArenaMemoryAllocator; import org.apache.druid.frame.channel.BlockingQueueFrameChannel; +import java.io.IOException; + /** * An {@link OutputChannelFactory} that generates {@link BlockingQueueFrameChannel}. */ @@ -46,6 +48,12 @@ public OutputChannel openChannel(final int partitionNumber) ); } + @Override + public PartitionedOutputChannel openChannel(String name, boolean deleteAfterRead) throws IOException + { + throw new UnsupportedOperationException("Opening in-memory partitioned channels is not supported"); + } + @Override public OutputChannel openNilChannel(final int partitionNumber) { diff --git a/processing/src/main/java/org/apache/druid/frame/processor/FileOutputChannelFactory.java b/processing/src/main/java/org/apache/druid/frame/processor/FileOutputChannelFactory.java index 533d3c9d66f3..dc1417458cb4 100644 --- a/processing/src/main/java/org/apache/druid/frame/processor/FileOutputChannelFactory.java +++ b/processing/src/main/java/org/apache/druid/frame/processor/FileOutputChannelFactory.java @@ -22,6 +22,7 @@ import com.google.common.base.Suppliers; import org.apache.druid.frame.Frame; import org.apache.druid.frame.allocation.ArenaMemoryAllocator; +import org.apache.druid.frame.channel.PartitionedReadableFrameChannel; import org.apache.druid.frame.channel.ReadableFileFrameChannel; import org.apache.druid.frame.channel.ReadableFrameChannel; import org.apache.druid.frame.channel.WritableFrameFileChannel; @@ -32,6 +33,7 @@ import java.io.File; import java.io.IOException; +import java.io.UncheckedIOException; import java.nio.ByteBuffer; import java.nio.file.Files; import java.nio.file.StandardOpenOption; @@ -92,6 +94,62 @@ public OutputChannel openChannel(int partitionNumber) throws IOException ); } + @Override + public PartitionedOutputChannel openChannel(String name, boolean deleteAfterRead) throws IOException + { + FileUtils.mkdirp(fileChannelsDirectory); + final File file = new File(fileChannelsDirectory, name); + WritableFrameFileChannel writableFrameFileChannel = new WritableFrameFileChannel( + FrameFileWriter.open( + Files.newByteChannel( + file.toPath(), + StandardOpenOption.CREATE_NEW, + StandardOpenOption.WRITE + ), + ByteBuffer.allocate(Frame.compressionBufferSize(frameSize)) + ) + ); + Supplier frameFileSupplier = Suppliers.memoize( + () -> { + try { + return deleteAfterRead ? FrameFile.open(file, FrameFile.Flag.DELETE_ON_CLOSE) + : FrameFile.open(file); + } + catch (IOException e) { + throw new UncheckedIOException(e); + } + } + )::get; + final Supplier partitionedReadableFrameChannelSupplier = Suppliers.memoize( + () -> new PartitionedReadableFrameChannel() + { + @Override + public ReadableFrameChannel openChannel(int partitionNumber) + { + FrameFile fileHandle = frameFileSupplier.get(); + fileHandle = fileHandle.newReference(); + return new ReadableFileFrameChannel( + fileHandle, + fileHandle.getPartitionStartFrame(partitionNumber), + fileHandle.getPartitionStartFrame(partitionNumber + 1) + ); + } + + @Override + public void close() throws IOException + { + frameFileSupplier.get().close(); + } + } + )::get; + + return PartitionedOutputChannel.pair( + writableFrameFileChannel, + ArenaMemoryAllocator.createOnHeap(frameSize), + partitionedReadableFrameChannelSupplier + ); + } + @Override public OutputChannel openNilChannel(final int partitionNumber) { diff --git a/processing/src/main/java/org/apache/druid/frame/processor/OutputChannelFactory.java b/processing/src/main/java/org/apache/druid/frame/processor/OutputChannelFactory.java index 11de551332e2..4b0ca79aad8f 100644 --- a/processing/src/main/java/org/apache/druid/frame/processor/OutputChannelFactory.java +++ b/processing/src/main/java/org/apache/druid/frame/processor/OutputChannelFactory.java @@ -31,6 +31,11 @@ public interface OutputChannelFactory */ OutputChannel openChannel(int partitionNumber) throws IOException; + /** + * Create a channel pair tagged with a particular name and a flag to delete the channel data after its read. + */ + PartitionedOutputChannel openChannel(String name, boolean deleteAfterRead) throws IOException; + /** * Create a non-writable, always-empty channel pair tagged with a particular partition number. * diff --git a/processing/src/main/java/org/apache/druid/frame/processor/PartitionedOutputChannel.java b/processing/src/main/java/org/apache/druid/frame/processor/PartitionedOutputChannel.java new file mode 100644 index 000000000000..6ea9d2909be4 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/frame/processor/PartitionedOutputChannel.java @@ -0,0 +1,117 @@ +/* + * 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.frame.processor; + +import com.google.common.base.Preconditions; +import com.google.common.base.Suppliers; +import org.apache.druid.frame.allocation.MemoryAllocator; +import org.apache.druid.frame.channel.PartitionedReadableFrameChannel; +import org.apache.druid.frame.channel.WritableFrameChannel; +import org.apache.druid.java.util.common.ISE; + +import javax.annotation.Nullable; +import java.util.function.Function; +import java.util.function.Supplier; + +public class PartitionedOutputChannel +{ + @Nullable + private final WritableFrameChannel writableChannel; + @Nullable + private final MemoryAllocator frameMemoryAllocator; + private final Supplier readableChannelSupplier; + + private PartitionedOutputChannel( + @Nullable final WritableFrameChannel writableChannel, + @Nullable final MemoryAllocator frameMemoryAllocator, + final Supplier readableChannelSupplier + ) + { + this.writableChannel = writableChannel; + this.frameMemoryAllocator = frameMemoryAllocator; + this.readableChannelSupplier = readableChannelSupplier; + } + + /** + * Creates a partitioned output channel pair. + * + * @param writableChannel writable channel for producer + * @param frameMemoryAllocator memory allocator for producer to use while writing frames to the channel + * @param readableChannelSupplier partitioned readable channel for consumer. May be called multiple times, so you + * should wrap this in {@link Suppliers#memoize} if needed. + */ + public static PartitionedOutputChannel pair( + final WritableFrameChannel writableChannel, + final MemoryAllocator frameMemoryAllocator, + final Supplier readableChannelSupplier + ) + { + return new PartitionedOutputChannel( + Preconditions.checkNotNull(writableChannel, "writableChannel"), + Preconditions.checkNotNull(frameMemoryAllocator, "frameMemoryAllocator"), + readableChannelSupplier + ); + } + + /** + * Returns the writable channel of this pair. The producer writes to this channel. + */ + public WritableFrameChannel getWritableChannel() + { + if (writableChannel == null) { + throw new ISE("Writable channel is not available"); + } else { + return writableChannel; + } + } + + /** + * Returns the memory allocator for the writable channel. The producer uses this to generate frames for the channel. + */ + public MemoryAllocator getFrameMemoryAllocator() + { + if (frameMemoryAllocator == null) { + throw new ISE("Writable channel is not available"); + } else { + return frameMemoryAllocator; + } + } + + /** + * Returns the partitioned readable channel supplier of this pair. The consumer reads from this channel. + */ + public Supplier getReadableChannelSupplier() + { + return readableChannelSupplier; + } + + public PartitionedOutputChannel mapWritableChannel(final Function mapFn) + { + if (writableChannel == null) { + return this; + } else { + return new PartitionedOutputChannel( + mapFn.apply(writableChannel), + frameMemoryAllocator, + readableChannelSupplier + ); + } + } +} diff --git a/processing/src/main/java/org/apache/druid/frame/processor/SuperSorter.java b/processing/src/main/java/org/apache/druid/frame/processor/SuperSorter.java index ad4942edd646..b4bb2c37ed5f 100644 --- a/processing/src/main/java/org/apache/druid/frame/processor/SuperSorter.java +++ b/processing/src/main/java/org/apache/druid/frame/processor/SuperSorter.java @@ -20,6 +20,7 @@ package org.apache.druid.frame.processor; import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.ImmutableList; import com.google.common.math.LongMath; import com.google.common.primitives.Ints; import com.google.common.util.concurrent.FutureCallback; @@ -40,12 +41,10 @@ import org.apache.druid.frame.allocation.MemoryAllocator; import org.apache.druid.frame.channel.BlockingQueueFrameChannel; import org.apache.druid.frame.channel.FrameWithPartition; -import org.apache.druid.frame.channel.ReadableFileFrameChannel; +import org.apache.druid.frame.channel.PartitionedReadableFrameChannel; import org.apache.druid.frame.channel.ReadableFrameChannel; import org.apache.druid.frame.channel.WritableFrameChannel; -import org.apache.druid.frame.channel.WritableFrameFileChannel; import org.apache.druid.frame.file.FrameFile; -import org.apache.druid.frame.file.FrameFileWriter; import org.apache.druid.frame.key.ClusterBy; import org.apache.druid.frame.key.ClusterByPartitions; import org.apache.druid.frame.read.FrameReader; @@ -57,12 +56,9 @@ import org.apache.druid.utils.CloseableUtils; import javax.annotation.Nullable; -import java.io.File; import java.io.IOException; +import java.io.UncheckedIOException; import java.math.RoundingMode; -import java.nio.ByteBuffer; -import java.nio.file.Files; -import java.nio.file.StandardOpenOption; import java.util.ArrayDeque; import java.util.ArrayList; import java.util.Arrays; @@ -73,7 +69,6 @@ import java.util.Queue; import java.util.concurrent.ExecutionException; import java.util.function.Consumer; -import java.util.function.Supplier; import java.util.stream.IntStream; /** @@ -126,9 +121,8 @@ public class SuperSorter private final ClusterBy clusterBy; private final ListenableFuture outputPartitionsFuture; private final FrameProcessorExecutor exec; - private final File directory; private final OutputChannelFactory outputChannelFactory; - private final Supplier innerFrameAllocatorMaker; + private final OutputChannelFactory intermediateOutputChannelFactory; private final int maxChannelsPerProcessor; private final int maxActiveProcessors; private final long rowLimit; @@ -148,6 +142,9 @@ public class SuperSorter @GuardedBy("runWorkersLock") private List outputChannels = null; + @GuardedBy("runWorkersLock") + private final Map levelAndRankToReadableChannelMap = new HashMap<>(); + @GuardedBy("runWorkersLock") private int activeProcessors = 0; @@ -169,9 +166,6 @@ public class SuperSorter @GuardedBy("runWorkersLock") private int ultimateMergersRunSoFar = 0; - @GuardedBy("runWorkersLock") - private final Map penultimateFrameFileCache = new HashMap<>(); - @GuardedBy("runWorkersLock") private SettableFuture allDone = null; @@ -187,34 +181,29 @@ public class SuperSorter /** * Initializes a SuperSorter. * - * @param inputChannels input channels. All frames in these channels must be sorted according to the - * {@link ClusterBy#getColumns()}, or else sorting will not produce correct - * output. - * @param frameReader frame reader for the input channels - * @param clusterBy desired sorting order - * @param outputPartitionsFuture a future that resolves to the desired output partitions. Sorting will block - * prior to writing out final outputs until this future resolves. However, the - * sorter will be able to read all inputs even if this future is unresolved. - * If output need not be partitioned, use - * {@link ClusterByPartitions#oneUniversalPartition()}. In this case a single - * sorted channel is generated. - * @param exec executor to perform work in - * @param temporaryDirectory directory to use for scratch files. This must have enough space to store at - * least two copies of the dataset in {@link FrameFile} format. - * @param outputChannelFactory factory for partitioned, sorted output channels - * @param innerFrameAllocatorMaker supplier for allocators that are used to make merged frames for intermediate - * levels of merging, prior to the final output. Final output frame allocation is - * controlled by outputChannelFactory. One allocator is created per intermediate - * scratch file. - * @param maxActiveProcessors maximum number of merging processors to execute at once in the provided - * {@link FrameProcessorExecutor} - * @param maxChannelsPerProcessor maximum number of channels to merge at once per merging processor - * @param rowLimit limit to apply during sorting. The limit is merely advisory: the actual number - * of rows returned may be larger than the limit. The limit is applied across - * all partitions, not to each partition individually. - * @param cancellationId cancellation id to use when running processors in the provided - * {@link FrameProcessorExecutor}. - * @param superSorterProgressTracker progress tracker + * @param inputChannels input channels. All frames in these channels must be sorted according to the + * {@link ClusterBy#getColumns()}, or else sorting will not produce correct + * output. + * @param frameReader frame reader for the input channels + * @param clusterBy desired sorting order + * @param outputPartitionsFuture a future that resolves to the desired output partitions. Sorting will block + * prior to writing out final outputs until this future resolves. However, the + * sorter will be able to read all inputs even if this future is unresolved. + * If output need not be partitioned, use + * {@link ClusterByPartitions#oneUniversalPartition()}. In this case a single + * sorted channel is generated. + * @param exec executor to perform work in + * @param outputChannelFactory factory for partitioned, sorted output channels + * @param intermediateOutputChannelFactory factory for intermediate data produced by sorting levels + * @param maxActiveProcessors maximum number of merging processors to execute at once in the provided + * {@link FrameProcessorExecutor} + * @param maxChannelsPerProcessor maximum number of channels to merge at once per merging processor + * @param rowLimit limit to apply during sorting. The limit is merely advisory: the actual number + * of rows returned may be larger than the limit. The limit is applied across + * all partitions, not to each partition individually. + * @param cancellationId cancellation id to use when running processors in the provided + * {@link FrameProcessorExecutor}. + * @param superSorterProgressTracker progress tracker */ public SuperSorter( final List inputChannels, @@ -222,9 +211,8 @@ public SuperSorter( final ClusterBy clusterBy, final ListenableFuture outputPartitionsFuture, final FrameProcessorExecutor exec, - final File temporaryDirectory, final OutputChannelFactory outputChannelFactory, - final Supplier innerFrameAllocatorMaker, + final OutputChannelFactory intermediateOutputChannelFactory, final int maxActiveProcessors, final int maxChannelsPerProcessor, final long rowLimit, @@ -237,9 +225,8 @@ public SuperSorter( this.clusterBy = clusterBy; this.outputPartitionsFuture = outputPartitionsFuture; this.exec = exec; - this.directory = temporaryDirectory; this.outputChannelFactory = outputChannelFactory; - this.innerFrameAllocatorMaker = innerFrameAllocatorMaker; + this.intermediateOutputChannelFactory = intermediateOutputChannelFactory; this.maxChannelsPerProcessor = maxChannelsPerProcessor; this.maxActiveProcessors = maxActiveProcessors; this.rowLimit = rowLimit; @@ -457,7 +444,7 @@ private boolean runNextLevelZeroMerger() in.add(singleReadableFrameChannel(new FrameWithPartition(frame, FrameWithPartition.NO_PARTITION))); } - runMerger(0, levelZeroMergersRunSoFar++, in, null); + runMerger(0, levelZeroMergersRunSoFar++, in, null, ImmutableList.of()); return true; } @@ -517,19 +504,26 @@ private boolean runNextMiddleMerger() (pos == maxChannelsPerProcessor - 1 || (totalInputs != UNKNOWN_TOTAL && w == totalInputs - 1))) { // We found a set to merge. Let's collect the input channels and launch the merger. final List in = new ArrayList<>(); + final List partitionedReadableFrameChannels = new ArrayList<>(); for (long i = currentSetStart; i < currentSetStart + maxChannelsPerProcessor; i++) { if (inputsReady.remove(i)) { - try { - final FrameFile handle = FrameFile.open(mergerOutputFile(inLevel, i), FrameFile.Flag.DELETE_ON_CLOSE); - in.add(new ReadableFileFrameChannel(handle)); - } - catch (IOException e) { - throw new RuntimeException(e); - } + String levelAndRankKey = StringUtils.format("merged.%d.%d", inLevel, i); + PartitionedReadableFrameChannel partitionedReadableFrameChannel = + levelAndRankToReadableChannelMap.remove(levelAndRankKey) + .getReadableChannelSupplier() + .get(); + in.add(partitionedReadableFrameChannel.openChannel(0)); + partitionedReadableFrameChannels.add(partitionedReadableFrameChannel); } } - runMerger(outLevel, currentSetStart / maxChannelsPerProcessor, in, outPartitions); + runMerger( + outLevel, + currentSetStart / maxChannelsPerProcessor, + in, + outPartitions, + partitionedReadableFrameChannels + ); return true; } else if (w == currentSetStart + currentSetIndex + 1) { currentSetIndex++; @@ -571,24 +565,11 @@ private boolean runNextUltimateMerger() final List in = new ArrayList<>(numInputs); for (long i = 0; i < numInputs; i++) { - final FrameFile fileHandle = penultimateFrameFileCache.computeIfAbsent( - mergerOutputFile(inLevel, i), - file -> { - try { - return FrameFile.open(file, FrameFile.Flag.DELETE_ON_CLOSE); - } - catch (IOException e) { - throw new RuntimeException(e); - } - } - ).newReference(); - in.add( - new ReadableFileFrameChannel( - fileHandle, - fileHandle.getPartitionStartFrame(ultimateMergersRunSoFar), - fileHandle.getPartitionStartFrame(ultimateMergersRunSoFar + 1) - ) + levelAndRankToReadableChannelMap.get(StringUtils.format("merged.%d.%d", inLevel, i)) + .getReadableChannelSupplier() + .get() + .openChannel(ultimateMergersRunSoFar) ); } @@ -596,7 +577,7 @@ private boolean runNextUltimateMerger() outputChannels = Arrays.asList(new OutputChannel[getOutputPartitions().size()]); } - runMerger(outLevel, ultimateMergersRunSoFar, in, null); + runMerger(outLevel, ultimateMergersRunSoFar, in, null, ImmutableList.of()); ultimateMergersRunSoFar++; return true; } @@ -606,12 +587,14 @@ private void runMerger( final int level, final long rank, final List in, - @Nullable final ClusterByPartitions partitions + @Nullable final ClusterByPartitions partitions, + List partitionedReadableChannelsToClose ) { try { final WritableFrameChannel writableChannel; final MemoryAllocator frameAllocator; + String levelAndRankKey = StringUtils.format("merged.%d.%d", level, rank); if (totalMergingLevels != UNKNOWN_LEVEL && level == totalMergingLevels - 1) { final int intRank = Ints.checkedCast(rank); @@ -620,17 +603,13 @@ private void runMerger( writableChannel = outputChannel.getWritableChannel(); frameAllocator = outputChannel.getFrameMemoryAllocator(); } else { - frameAllocator = innerFrameAllocatorMaker.get(); - writableChannel = new WritableFrameFileChannel( - FrameFileWriter.open( - Files.newByteChannel( - mergerOutputFile(level, rank).toPath(), - StandardOpenOption.CREATE_NEW, - StandardOpenOption.WRITE - ), - ByteBuffer.allocate(Frame.compressionBufferSize(frameAllocator.capacity())) - ) + PartitionedOutputChannel partitionedOutputChannel = intermediateOutputChannelFactory.openChannel( + levelAndRankKey, + true ); + writableChannel = partitionedOutputChannel.getWritableChannel(); + frameAllocator = partitionedOutputChannel.getFrameMemoryAllocator(); + levelAndRankToReadableChannelMap.put(levelAndRankKey, partitionedOutputChannel); } final FrameChannelMerger worker = @@ -642,7 +621,6 @@ private void runMerger( FrameType.ROW_BASED, // Row-based frames are generally preferred as inputs to mergers frameAllocator, frameReader.signature(), - // No sortColumns, because FrameChannelMerger generates frames that are sorted all on its own Collections.emptyList() ), @@ -656,6 +634,14 @@ private void runMerger( outputsReadyByLevel.computeIfAbsent(level, ignored2 -> new LongRBTreeSet()) .add(rank); superSorterProgressTracker.addMergedBatchesForLevel(level, 1); + for (PartitionedReadableFrameChannel partitionedReadableFrameChannel : partitionedReadableChannelsToClose) { + try { + partitionedReadableFrameChannel.close(); + } + catch (IOException e) { + log.warn(e, "Could not close channel for level [%d] and rank [%d]", level, rank); + } + } } }); } @@ -811,15 +797,16 @@ private void cleanUp() outputsReadyByLevel.clear(); inputBuffer.clear(); - - for (FrameFile frameFile : penultimateFrameFileCache.values()) { - CloseableUtils.closeAndSuppressExceptions( - frameFile, - e -> log.warn(e, "Could not close intermediate file [%s]", frameFile.file()) - ); + for (Map.Entry cleanupEntry : + levelAndRankToReadableChannelMap.entrySet()) { + try { + cleanupEntry.getValue().getReadableChannelSupplier().get().close(); + } + catch (IOException e) { + throw new UncheckedIOException("Unable to close channel for name : " + cleanupEntry.getKey(), e); + } } - - penultimateFrameFileCache.clear(); + levelAndRankToReadableChannelMap.clear(); if (!inputChannelsToRead.isEmpty()) { for (final ReadableFrameChannel inputChannel : inputChannels) { @@ -835,11 +822,6 @@ private void cleanUp() inputChannelsToRead.clear(); } - private File mergerOutputFile(final int level, final long rank) - { - return new File(directory, StringUtils.format("merged.%d.%d", level, rank)); - } - /** * Returns a string encapsulating the current state of this object. */ diff --git a/processing/src/test/java/org/apache/druid/frame/channel/ReadableByteChunksFrameChannelTest.java b/processing/src/test/java/org/apache/druid/frame/channel/ReadableByteChunksFrameChannelTest.java index a6e95f34ab85..2c96be3c1ca8 100644 --- a/processing/src/test/java/org/apache/druid/frame/channel/ReadableByteChunksFrameChannelTest.java +++ b/processing/src/test/java/org/apache/druid/frame/channel/ReadableByteChunksFrameChannelTest.java @@ -69,7 +69,7 @@ public static class NonParameterizedTests extends InitializedNullHandlingTest @Test public void testZeroBytes() { - final ReadableByteChunksFrameChannel channel = ReadableByteChunksFrameChannel.create("test"); + final ReadableByteChunksFrameChannel channel = ReadableByteChunksFrameChannel.create("test", false); channel.doneWriting(); Assert.assertTrue(channel.canRead()); @@ -85,7 +85,7 @@ public void testZeroBytes() @Test public void testZeroBytesWithSpecialError() { - final ReadableByteChunksFrameChannel channel = ReadableByteChunksFrameChannel.create("test"); + final ReadableByteChunksFrameChannel channel = ReadableByteChunksFrameChannel.create("test", false); channel.setError(new IllegalArgumentException("test error")); channel.doneWriting(); @@ -105,7 +105,7 @@ public void testEmptyFrameFile() throws IOException // File with no frames (but still well-formed). final File file = FrameTestUtil.writeFrameFile(Sequences.empty(), temporaryFolder.newFile()); - final ReadableByteChunksFrameChannel channel = ReadableByteChunksFrameChannel.create("test"); + final ReadableByteChunksFrameChannel channel = ReadableByteChunksFrameChannel.create("test", false); channel.addChunk(Files.toByteArray(file)); channel.doneWriting(); Assert.assertEquals(file.length(), channel.getBytesAdded()); @@ -143,7 +143,7 @@ public void testTruncatedFrameFile() throws IOException ByteStreams.readFully(in, truncatedFile); } - final ReadableByteChunksFrameChannel channel = ReadableByteChunksFrameChannel.create("test"); + final ReadableByteChunksFrameChannel channel = ReadableByteChunksFrameChannel.create("test", false); channel.addChunk(truncatedFile); channel.doneWriting(); Assert.assertEquals(truncatedFile.length, channel.getBytesAdded()); @@ -184,7 +184,7 @@ public void testSetError() throws IOException temporaryFolder.newFile() ); - final ReadableByteChunksFrameChannel channel = ReadableByteChunksFrameChannel.create("test"); + final ReadableByteChunksFrameChannel channel = ReadableByteChunksFrameChannel.create("test", false); final byte[] fileBytes = Files.toByteArray(file); final byte[] chunk1 = new byte[errorAtBytePosition]; System.arraycopy(fileBytes, 0, chunk1, 0, chunk1.length); @@ -252,7 +252,7 @@ public void testWriteFullyThenRead() throws IOException temporaryFolder.newFile() ); - final ReadableByteChunksFrameChannel channel = ReadableByteChunksFrameChannel.create("test"); + final ReadableByteChunksFrameChannel channel = ReadableByteChunksFrameChannel.create("test", false); ListenableFuture firstBackpressureFuture = null; long totalSize = 0; @@ -307,7 +307,7 @@ public void testWriteReadInterleaved() throws IOException temporaryFolder.newFile() ); - final ReadableByteChunksFrameChannel channel = ReadableByteChunksFrameChannel.create("test"); + final ReadableByteChunksFrameChannel channel = ReadableByteChunksFrameChannel.create("test", false); final BlockingQueueFrameChannel outChannel = new BlockingQueueFrameChannel(10_000); // Enough to hold all frames ListenableFuture backpressureFuture = null; diff --git a/processing/src/test/java/org/apache/druid/frame/channel/ReadableInputStreamFrameChannelTest.java b/processing/src/test/java/org/apache/druid/frame/channel/ReadableInputStreamFrameChannelTest.java index c019728c3cac..b3c0ec205aab 100644 --- a/processing/src/test/java/org/apache/druid/frame/channel/ReadableInputStreamFrameChannelTest.java +++ b/processing/src/test/java/org/apache/druid/frame/channel/ReadableInputStreamFrameChannelTest.java @@ -71,7 +71,8 @@ public void testSimpleFrameFile() ReadableInputStreamFrameChannel readableInputStreamFrameChannel = ReadableInputStreamFrameChannel.open( inputStream, "readSimpleFrameFile", - executorService + executorService, + false ); FrameTestUtil.assertRowsEqual( @@ -94,7 +95,8 @@ public void testEmptyFrameFile() throws IOException ReadableInputStreamFrameChannel readableInputStreamFrameChannel = ReadableInputStreamFrameChannel.open( Files.newInputStream(file.toPath()), "readEmptyFrameFile", - executorService + executorService, + false ); Assert.assertEquals(FrameTestUtil.readRowsFromFrameChannel( @@ -116,7 +118,8 @@ public void testZeroBytesFrameFile() throws IOException ReadableInputStreamFrameChannel readableInputStreamFrameChannel = ReadableInputStreamFrameChannel.open( Files.newInputStream(file.toPath()), "testZeroBytesFrameFile", - executorService + executorService, + false ); final IllegalStateException e = Assert.assertThrows( @@ -161,7 +164,8 @@ public void testTruncatedFrameFile() throws IOException ReadableInputStreamFrameChannel readableInputStreamFrameChannel = ReadableInputStreamFrameChannel.open( new ByteArrayInputStream(truncatedFile), "readTruncatedFrameFile", - executorService + executorService, + false ); expectedException.expect(ISE.class); @@ -186,7 +190,8 @@ public void testIncorrectFrameFile() throws IOException ReadableInputStreamFrameChannel readableInputStreamFrameChannel = ReadableInputStreamFrameChannel.open( Files.newInputStream(file.toPath()), "readIncorrectFrameFile", - executorService + executorService, + false ); expectedException.expect(ISE.class); @@ -209,7 +214,8 @@ public void closeInputStreamWhileReading() throws IOException ReadableInputStreamFrameChannel readableInputStreamFrameChannel = ReadableInputStreamFrameChannel.open( inputStream, "closeInputStreamWhileReading", - executorService + executorService, + false ); inputStream.close(); @@ -233,7 +239,8 @@ public void closeInputStreamWhileReadingCheckError() throws IOException, Interru ReadableInputStreamFrameChannel readableInputStreamFrameChannel = ReadableInputStreamFrameChannel.open( inputStream, "closeInputStreamWhileReadingCheckError", - executorService + executorService, + false ); inputStream.close(); diff --git a/processing/src/test/java/org/apache/druid/frame/file/FrameFileHttpResponseHandlerTest.java b/processing/src/test/java/org/apache/druid/frame/file/FrameFileHttpResponseHandlerTest.java index c6c2dc198dcb..4eeaaddbe892 100644 --- a/processing/src/test/java/org/apache/druid/frame/file/FrameFileHttpResponseHandlerTest.java +++ b/processing/src/test/java/org/apache/druid/frame/file/FrameFileHttpResponseHandlerTest.java @@ -104,7 +104,7 @@ public void setUp() throws IOException temporaryFolder.newFile() ); - channel = ReadableByteChunksFrameChannel.create("test"); + channel = ReadableByteChunksFrameChannel.create("test", false); handler = new FrameFileHttpResponseHandler(channel); } diff --git a/processing/src/test/java/org/apache/druid/frame/processor/SuperSorterTest.java b/processing/src/test/java/org/apache/druid/frame/processor/SuperSorterTest.java index 77c56f0a2c04..c678e676c880 100644 --- a/processing/src/test/java/org/apache/druid/frame/processor/SuperSorterTest.java +++ b/processing/src/test/java/org/apache/druid/frame/processor/SuperSorterTest.java @@ -27,6 +27,7 @@ import org.apache.druid.frame.FrameType; import org.apache.druid.frame.allocation.ArenaMemoryAllocator; import org.apache.druid.frame.channel.BlockingQueueFrameChannel; +import org.apache.druid.frame.channel.DurableStorageOutputChannelFactory; import org.apache.druid.frame.channel.ReadableFileFrameChannel; import org.apache.druid.frame.channel.ReadableFrameChannel; import org.apache.druid.frame.channel.WritableFrameChannel; @@ -54,6 +55,7 @@ import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.storage.local.LocalFileStorageConnector; import org.apache.druid.testing.InitializedNullHandlingTest; import org.junit.After; import org.junit.Assert; @@ -110,7 +112,18 @@ public void tearDown() } @Test - public void testSingleEmptyInputChannel() throws Exception + public void testSingleEmptyInputChannel_fileStorage() throws Exception + { + testSingleEmptyInputChannel(false); + } + + @Test + public void testSingleEmptyInputChannel_durableStorage() throws Exception + { + testSingleEmptyInputChannel(true); + } + + private void testSingleEmptyInputChannel(boolean isDurableStorage) throws Exception { final BlockingQueueFrameChannel inputChannel = BlockingQueueFrameChannel.minimal(); inputChannel.writable().close(); @@ -118,15 +131,23 @@ public void testSingleEmptyInputChannel() throws Exception final SettableFuture outputPartitionsFuture = SettableFuture.create(); final SuperSorterProgressTracker superSorterProgressTracker = new SuperSorterProgressTracker(); + final File tempFolder = temporaryFolder.newFolder(); final SuperSorter superSorter = new SuperSorter( Collections.singletonList(inputChannel.readable()), FrameReader.create(RowSignature.empty()), ClusterBy.none(), outputPartitionsFuture, exec, - temporaryFolder.newFolder(), - new FileOutputChannelFactory(temporaryFolder.newFolder(), FRAME_SIZE), - () -> ArenaMemoryAllocator.createOnHeap(FRAME_SIZE), + new FileOutputChannelFactory(tempFolder, FRAME_SIZE), + isDurableStorage ? new DurableStorageOutputChannelFactory( + "0", + 0, + 0, + "0", + FRAME_SIZE, + new LocalFileStorageConnector(tempFolder), + tempFolder + ) : new FileOutputChannelFactory(tempFolder, FRAME_SIZE), 2, 2, -1, @@ -271,7 +292,8 @@ private void setUpInputChannels(final ClusterBy clusterBy) throws Exception private OutputChannels verifySuperSorter( final ClusterBy clusterBy, - final ClusterByPartitions clusterByPartitions + final ClusterByPartitions clusterByPartitions, + boolean isDurableStorage ) throws Exception { final RowKeyReader keyReader = clusterBy.keyReader(signature); @@ -279,15 +301,23 @@ private OutputChannels verifySuperSorter( final SettableFuture clusterByPartitionsFuture = SettableFuture.create(); final SuperSorterProgressTracker superSorterProgressTracker = new SuperSorterProgressTracker(); + File tempFolder = temporaryFolder.newFolder(); final SuperSorter superSorter = new SuperSorter( inputChannels, frameReader, clusterBy, clusterByPartitionsFuture, exec, - temporaryFolder.newFolder(), - new FileOutputChannelFactory(temporaryFolder.newFolder(), maxBytesPerFrame), - () -> ArenaMemoryAllocator.createOnHeap(maxBytesPerFrame), + new FileOutputChannelFactory(tempFolder, maxBytesPerFrame), + isDurableStorage ? new DurableStorageOutputChannelFactory( + "0", + 0, + 0, + "0", + maxBytesPerFrame, + new LocalFileStorageConnector(tempFolder), + tempFolder + ) : new FileOutputChannelFactory(tempFolder, maxBytesPerFrame), maxActiveProcessors, maxChannelsPerProcessor, -1, @@ -387,7 +417,7 @@ public void test_clusterByQualityLongAscRowNumberAsc_onePartition() throws Excep ); setUpInputChannels(clusterBy); - verifySuperSorter(clusterBy, ClusterByPartitions.oneUniversalPartition()); + verifySuperSorter(clusterBy, ClusterByPartitions.oneUniversalPartition(), false); } @Test @@ -411,7 +441,8 @@ public void test_clusterByQualityLongAscRowNumberAsc_twoPartitionsOneEmpty() thr new ClusterByPartition(null, zeroZero), // empty partition new ClusterByPartition(zeroZero, null) // all data goes in here ) - ) + ), + false ); // Verify that one of the partitions is actually empty. @@ -473,7 +504,7 @@ public void test_clusterByQualityDescRowNumberAsc_fourPartitions() throws Except Assert.assertEquals(4, partitions.size()); - verifySuperSorter(clusterBy, partitions); + verifySuperSorter(clusterBy, partitions, false); } @Test @@ -513,7 +544,7 @@ public void test_clusterByTimeAscMarketAscRowNumberAsc_fourPartitions() throws E Assert.assertEquals(4, partitions.size()); - verifySuperSorter(clusterBy, partitions); + verifySuperSorter(clusterBy, partitions, false); } @Test @@ -552,7 +583,7 @@ public void test_clusterByPlacementishDescRowNumberAsc_fourPartitions() throws E Assert.assertEquals(4, partitions.size()); - verifySuperSorter(clusterBy, partitions); + verifySuperSorter(clusterBy, partitions, false); } @Test @@ -591,7 +622,46 @@ public void test_clusterByQualityLongDescRowNumberAsc_fourPartitions() throws Ex Assert.assertEquals(4, partitions.size()); - verifySuperSorter(clusterBy, partitions); + verifySuperSorter(clusterBy, partitions, false); + } + + @Test + public void test_clusterByQualityLongDescRowNumberAsc_fourPartitions_durableStorage() throws Exception + { + final ClusterBy clusterBy = new ClusterBy( + ImmutableList.of( + new SortColumn("qualityLong", true), + new SortColumn(FrameTestUtil.ROW_NUMBER_COLUMN, false) + ), + 0 + ); + + setUpInputChannels(clusterBy); + + final ClusterByPartitions partitions = new ClusterByPartitions( + ImmutableList.of( + new ClusterByPartition( + createKey(clusterBy, 1800L, 8L), + createKey(clusterBy, 1600L, 506L) + ), + new ClusterByPartition( + createKey(clusterBy, 1600L, 506L), + createKey(clusterBy, 1400L, 204L) + ), + new ClusterByPartition( + createKey(clusterBy, 1400L, 204L), + createKey(clusterBy, 1300L, 900L) + ), + new ClusterByPartition( + createKey(clusterBy, 1300L, 900L), + null + ) + ) + ); + + Assert.assertEquals(4, partitions.size()); + + verifySuperSorter(clusterBy, partitions, true); } private RowKey createKey(final ClusterBy clusterBy, final Object... objects) From 246620a1ab26dac251155421ac9572ea6a0352b0 Mon Sep 17 00:00:00 2001 From: Rohan Garg Date: Wed, 16 Nov 2022 12:37:06 +0530 Subject: [PATCH 02/15] Fix empty reads --- .../DurableStoragePartitionedReadableFrameChannel.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/processing/src/main/java/org/apache/druid/frame/channel/DurableStoragePartitionedReadableFrameChannel.java b/processing/src/main/java/org/apache/druid/frame/channel/DurableStoragePartitionedReadableFrameChannel.java index ae3f529c8d92..6d20b73d9fc6 100644 --- a/processing/src/main/java/org/apache/druid/frame/channel/DurableStoragePartitionedReadableFrameChannel.java +++ b/processing/src/main/java/org/apache/druid/frame/channel/DurableStoragePartitionedReadableFrameChannel.java @@ -60,8 +60,12 @@ public ReadableFrameChannel openChannel(int partitionNumber) int startFrame = frameFileFooter.getPartitionStartFrame(partitionNumber); int endFrame = frameFileFooter.getPartitionStartFrame(partitionNumber + 1); long startByte = startFrame == 0 ? FrameFileWriter.MAGIC.length : frameFileFooter.getFrameEndPosition(startFrame - 1); - long endByte = frameFileFooter.getFrameEndPosition(endFrame - 1); + long endByte = endFrame == 0 ? FrameFileWriter.MAGIC.length : frameFileFooter.getFrameEndPosition(endFrame - 1); + long size = endByte - startByte; + if (size <= 0) { + return ReadableNilFrameChannel.INSTANCE; + } try { return ReadableInputStreamFrameChannel.open( storageConnector.readRange(frameFileFullPath, startByte, endByte - startByte), From e1cda34684205efe9bd2c253bb7921cb1b77544f Mon Sep 17 00:00:00 2001 From: Rohan Garg Date: Tue, 22 Nov 2022 15:27:22 +0530 Subject: [PATCH 03/15] Review fixes --- .../apache/druid/msq/exec/ControllerImpl.java | 16 +++++++++------- .../indexing/CountingOutputChannelFactory.java | 6 ++++-- .../indexing/CountingWritableFrameChannel.java | 7 +++++-- .../DurableStorageInputChannelFactory.java | 2 +- ...leStoragePartitionedReadableFrameChannel.java | 2 +- .../BlockingQueueOutputChannelFactory.java | 4 +--- 6 files changed, 21 insertions(+), 16 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java index 0588ffe9dc54..feb72c5a7bd9 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java @@ -156,6 +156,7 @@ import org.apache.druid.msq.util.MultiStageQueryContext; import org.apache.druid.msq.util.PassthroughAggregatorFactory; import org.apache.druid.query.Query; +import org.apache.druid.query.QueryContext; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.groupby.GroupByQuery; import org.apache.druid.query.groupby.GroupByQueryConfig; @@ -268,6 +269,7 @@ public class ControllerImpl implements Controller private volatile WorkerClient netClient; private volatile FaultsExceededChecker faultsExceededChecker = null; + private final boolean isDurableStageStorageEnabled; public ControllerImpl( final MSQControllerTask task, @@ -276,6 +278,9 @@ public ControllerImpl( { this.task = task; this.context = context; + this.isDurableStageStorageEnabled = MultiStageQueryContext.isDurableShuffleStorageEnabled( + QueryContext.of(task.getContext()) + ); } @Override @@ -521,9 +526,6 @@ private QueryDefinition initializeQueryDefAndState(final Closer closer) this.netClient = new ExceptionWrappingWorkerClient(context.taskClientFor(this)); closer.register(netClient::close); - final boolean isDurableStorageEnabled = - MultiStageQueryContext.isDurableShuffleStorageEnabled(task.getQuerySpec().getQuery().context()); - final QueryDefinition queryDef = makeQueryDefinition( id(), makeQueryControllerToolKit(), @@ -534,7 +536,7 @@ private QueryDefinition initializeQueryDefAndState(final Closer closer) QueryValidator.validateQueryDef(queryDef); queryDefRef.set(queryDef); - log.debug("Query [%s] durable storage mode is set to %s.", queryDef.getQueryId(), isDurableStorageEnabled); + log.debug("Query [%s] durable storage mode is set to %s.", queryDef.getQueryId(), isDurableStageStorageEnabled); long maxParseExceptions = -1; @@ -551,7 +553,7 @@ private QueryDefinition initializeQueryDefAndState(final Closer closer) id(), task.getDataSource(), context, - isDurableStorageEnabled, + isDurableStageStorageEnabled, maxParseExceptions, // 10 minutes +- 2 minutes jitter TimeUnit.SECONDS.toMillis(600 + ThreadLocalRandom.current().nextInt(-4, 5) * 30L) @@ -1187,7 +1189,7 @@ private Yielder getFinalResultsYielder( final InputChannelFactory inputChannelFactory; - if (MultiStageQueryContext.isDurableShuffleStorageEnabled(task.getQuerySpec().getQuery().context())) { + if (isDurableStageStorageEnabled) { inputChannelFactory = DurableStorageInputChannelFactory.createStandardImplementation( id(), MSQTasks.makeStorageConnector(context.injector()), @@ -1289,7 +1291,7 @@ private void publishSegmentsIfNeeded( */ private void cleanUpDurableStorageIfNeeded() { - if (MultiStageQueryContext.isDurableShuffleStorageEnabled(task.getQuerySpec().getQuery().context())) { + if (isDurableStageStorageEnabled) { final String controllerDirName = DurableStorageUtils.getControllerDirectory(task.getId()); try { // Delete all temporary files as a failsafe diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/CountingOutputChannelFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/CountingOutputChannelFactory.java index 9938948f9a0c..7c905e589a6a 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/CountingOutputChannelFactory.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/CountingOutputChannelFactory.java @@ -50,7 +50,8 @@ public OutputChannel openChannel(int partitionNumber) throws IOException baseWritableChannel -> new CountingWritableFrameChannel( baseChannel.getWritableChannel(), - channelCounters + channelCounters, + partitionNumber ) ); } @@ -64,7 +65,8 @@ public PartitionedOutputChannel openChannel(String name, boolean deleteAfterRead baseWritableChannel -> new CountingWritableFrameChannel( baseChannel.getWritableChannel(), - channelCounters + channelCounters, + null ) ); } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/CountingWritableFrameChannel.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/CountingWritableFrameChannel.java index 1ee3ac69232d..c0df5107323f 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/CountingWritableFrameChannel.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/CountingWritableFrameChannel.java @@ -31,21 +31,24 @@ public class CountingWritableFrameChannel implements WritableFrameChannel { private final WritableFrameChannel baseChannel; private final ChannelCounters channelCounters; + private final Integer partitionNumber; public CountingWritableFrameChannel( final WritableFrameChannel baseChannel, - final ChannelCounters channelCounters + final ChannelCounters channelCounters, + final Integer partitionNumber ) { this.baseChannel = baseChannel; this.channelCounters = channelCounters; + this.partitionNumber = partitionNumber; } @Override public void write(FrameWithPartition frame) throws IOException { baseChannel.write(frame); - channelCounters.addFrame(frame.partition(), frame.frame()); + channelCounters.addFrame(partitionNumber == null ? frame.partition() : partitionNumber, frame.frame()); } @Override diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/DurableStorageInputChannelFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/DurableStorageInputChannelFactory.java index 5645a00f5bab..17eadcedb7dc 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/DurableStorageInputChannelFactory.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/DurableStorageInputChannelFactory.java @@ -133,7 +133,7 @@ public ReadableFrameChannel openChannel(StageId stageId, int workerNumber, int p * 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)} + * More information at {@link org.apache.druid.frame.channel.DurableStorageOutputChannelFactory#createSuccessFile(String)} */ public String findSuccessfulPartitionOutput( final String controllerTaskId, diff --git a/processing/src/main/java/org/apache/druid/frame/channel/DurableStoragePartitionedReadableFrameChannel.java b/processing/src/main/java/org/apache/druid/frame/channel/DurableStoragePartitionedReadableFrameChannel.java index 6d20b73d9fc6..3b3446f61217 100644 --- a/processing/src/main/java/org/apache/druid/frame/channel/DurableStoragePartitionedReadableFrameChannel.java +++ b/processing/src/main/java/org/apache/druid/frame/channel/DurableStoragePartitionedReadableFrameChannel.java @@ -80,7 +80,7 @@ public ReadableFrameChannel openChannel(int partitionNumber) } @Override - public void close() throws IOException + public void close() { try { storageConnector.deleteFile(frameFileFullPath); diff --git a/processing/src/main/java/org/apache/druid/frame/processor/BlockingQueueOutputChannelFactory.java b/processing/src/main/java/org/apache/druid/frame/processor/BlockingQueueOutputChannelFactory.java index 4bd3005307da..b7e3b4706ad7 100644 --- a/processing/src/main/java/org/apache/druid/frame/processor/BlockingQueueOutputChannelFactory.java +++ b/processing/src/main/java/org/apache/druid/frame/processor/BlockingQueueOutputChannelFactory.java @@ -22,8 +22,6 @@ import org.apache.druid.frame.allocation.ArenaMemoryAllocator; import org.apache.druid.frame.channel.BlockingQueueFrameChannel; -import java.io.IOException; - /** * An {@link OutputChannelFactory} that generates {@link BlockingQueueFrameChannel}. */ @@ -49,7 +47,7 @@ public OutputChannel openChannel(final int partitionNumber) } @Override - public PartitionedOutputChannel openChannel(String name, boolean deleteAfterRead) throws IOException + public PartitionedOutputChannel openChannel(String name, boolean deleteAfterRead) { throw new UnsupportedOperationException("Opening in-memory partitioned channels is not supported"); } From 9a04190045266a2c48a0192b0b96701cffda40b2 Mon Sep 17 00:00:00 2001 From: Rohan Garg Date: Tue, 22 Nov 2022 18:20:10 +0530 Subject: [PATCH 04/15] parameterize tests --- .travis.yml | 2 +- .../org/apache/druid/msq/exec/WorkerImpl.java | 6 +- .../msq/util/MultiStageQueryContext.java | 20 ------ .../msq/util/MultiStageQueryContextTest.java | 31 -------- .../frame/processor/SuperSorterTest.java | 72 ++++++++++--------- 5 files changed, 41 insertions(+), 90 deletions(-) diff --git a/.travis.yml b/.travis.yml index 750d9e7ce3b6..516f5ec16ffc 100644 --- a/.travis.yml +++ b/.travis.yml @@ -200,7 +200,7 @@ jobs: ${MAVEN_SKIP} -Dremoteresources.skip=true -Ddruid.generic.useDefaultValueForNull=${DRUID_USE_DEFAULT_VALUE_FOR_NULL} - sh -c "dmesg | egrep -i '(oom|out of memory|kill process|killed).*' -C 1 || exit 0" - free -m - - ${MVN} -pl ${MAVEN_PROJECTS} jacoco:report + - travis_wait 15 ${MVN} -pl ${MAVEN_PROJECTS} jacoco:report # Add merge target branch to determine diff (see https://github.com/travis-ci/travis-ci/issues/6069). # This is not needed for build triggered by tags, since there will be no code diff. - echo "TRAVIS_BRANCH=${TRAVIS_BRANCH}" # for debugging diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java index 9206501ac720..12b08891c34f 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java @@ -160,7 +160,6 @@ public class WorkerImpl implements Worker private final ConcurrentHashMap> stageOutputs = new ConcurrentHashMap<>(); private final ConcurrentHashMap stageCounters = new ConcurrentHashMap<>(); private final boolean durableStageStorageEnabled; - private final boolean durableTaskIntermediateStorageEnabled; /** * Set once in {@link #runTask} and never reassigned. @@ -187,9 +186,6 @@ public WorkerImpl(MSQWorkerTask task, WorkerContext context) this.durableStageStorageEnabled = MultiStageQueryContext.isDurableShuffleStorageEnabled( QueryContext.of(task.getContext()) ); - this.durableTaskIntermediateStorageEnabled = MultiStageQueryContext.isDurableTaskIntermediateStorageEnabled( - QueryContext.of(task.getContext()) - ); } @Override @@ -634,7 +630,7 @@ private OutputChannelFactory makeSuperSorterIntermediateOutputChannelFactory( { final int frameSize = frameContext.memoryParameters().getLargeFrameSize(); - if (durableTaskIntermediateStorageEnabled) { + if (durableStageStorageEnabled) { return DurableStorageOutputChannelFactory.createStandardImplementation( task.getControllerTaskId(), task().getWorkerNumber(), diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MultiStageQueryContext.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MultiStageQueryContext.java index fc9dbb519f66..2e7b1cce34f3 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MultiStageQueryContext.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MultiStageQueryContext.java @@ -25,7 +25,6 @@ import com.google.common.annotations.VisibleForTesting; import com.opencsv.RFC4180Parser; import com.opencsv.RFC4180ParserBuilder; -import org.apache.druid.java.util.common.IAE; import org.apache.druid.msq.kernel.WorkerAssignmentStrategy; import org.apache.druid.msq.sql.MSQMode; import org.apache.druid.query.QueryContext; @@ -61,9 +60,6 @@ public class MultiStageQueryContext public static final String CTX_ENABLE_DURABLE_SHUFFLE_STORAGE = "durableShuffleStorage"; private static final boolean DEFAULT_ENABLE_DURABLE_SHUFFLE_STORAGE = false; - public static final String CTX_ENABLE_DURABLE_TASK_INTERMEDIATE_STORAGE = "durableTaskIntermediateStorage"; - private static final boolean DEFAULT_ENABLE_DURABLE_TASK_INTERMEDIATE_STORAGE = false; - public static final String CTX_DESTINATION = "destination"; private static final String DEFAULT_DESTINATION = null; @@ -97,22 +93,6 @@ public static boolean isDurableShuffleStorageEnabled(final QueryContext queryCon ); } - public static boolean isDurableTaskIntermediateStorageEnabled(final QueryContext queryContext) - { - boolean intermediateStorageEnabled = queryContext.getBoolean( - CTX_ENABLE_DURABLE_TASK_INTERMEDIATE_STORAGE, - DEFAULT_ENABLE_DURABLE_TASK_INTERMEDIATE_STORAGE - ); - if (!isDurableShuffleStorageEnabled(queryContext) && intermediateStorageEnabled) { - throw new IAE( - "Cannot enable the feature %s without enabling %s", - CTX_ENABLE_DURABLE_TASK_INTERMEDIATE_STORAGE, - CTX_ENABLE_DURABLE_SHUFFLE_STORAGE - ); - } - return intermediateStorageEnabled; - } - public static boolean isFinalizeAggregations(final QueryContext queryContext) { return queryContext.getBoolean( diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/util/MultiStageQueryContextTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/util/MultiStageQueryContextTest.java index 27273b0251a3..befc4657c57c 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/util/MultiStageQueryContextTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/util/MultiStageQueryContextTest.java @@ -22,7 +22,6 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; -import org.apache.druid.java.util.common.IAE; import org.apache.druid.msq.kernel.WorkerAssignmentStrategy; import org.apache.druid.query.BadQueryContextException; import org.apache.druid.query.QueryContext; @@ -42,7 +41,6 @@ import static org.apache.druid.msq.util.MultiStageQueryContext.CTX_DESTINATION; import static org.apache.druid.msq.util.MultiStageQueryContext.CTX_ENABLE_DURABLE_SHUFFLE_STORAGE; -import static org.apache.druid.msq.util.MultiStageQueryContext.CTX_ENABLE_DURABLE_TASK_INTERMEDIATE_STORAGE; import static org.apache.druid.msq.util.MultiStageQueryContext.CTX_FINALIZE_AGGREGATIONS; import static org.apache.druid.msq.util.MultiStageQueryContext.CTX_MAX_NUM_TASKS; import static org.apache.druid.msq.util.MultiStageQueryContext.CTX_MSQ_MODE; @@ -67,35 +65,6 @@ public void isDurableShuffleStorageEnabled_parameterSetReturnsCorrectValue() Assert.assertTrue(MultiStageQueryContext.isDurableShuffleStorageEnabled(QueryContext.of(propertyMap))); } - @Test - public void isDurableTaskIntermediateStorageEnabled_noParameterSetReturnsDefaultValue() - { - Assert.assertFalse(MultiStageQueryContext.isDurableTaskIntermediateStorageEnabled(QueryContext.empty())); - } - - @Test - public void isDurableTaskIntermediateStorageEnabled_parameterSetReturnsCorrectValue() - { - Map propertyMap = ImmutableMap.of( - CTX_ENABLE_DURABLE_SHUFFLE_STORAGE, "true", - CTX_ENABLE_DURABLE_TASK_INTERMEDIATE_STORAGE, "true" - ); - Assert.assertTrue(MultiStageQueryContext.isDurableTaskIntermediateStorageEnabled(QueryContext.of(propertyMap))); - } - - @Test - public void isDurableIntermediateStorageEnabled_partialParameterSetThrows() - { - Map propertyMap = ImmutableMap.of( - CTX_ENABLE_DURABLE_SHUFFLE_STORAGE, "false", - CTX_ENABLE_DURABLE_TASK_INTERMEDIATE_STORAGE, "true" - ); - Assert.assertThrows( - IAE.class, - () -> MultiStageQueryContext.isDurableTaskIntermediateStorageEnabled(QueryContext.of(propertyMap)) - ); - } - @Test public void isFinalizeAggregations_noParameterSetReturnsDefaultValue() { diff --git a/processing/src/test/java/org/apache/druid/frame/processor/SuperSorterTest.java b/processing/src/test/java/org/apache/druid/frame/processor/SuperSorterTest.java index c678e676c880..553d0c340711 100644 --- a/processing/src/test/java/org/apache/druid/frame/processor/SuperSorterTest.java +++ b/processing/src/test/java/org/apache/druid/frame/processor/SuperSorterTest.java @@ -182,6 +182,7 @@ public static class ParameterizedCasesTest extends InitializedNullHandlingTest private final int maxActiveProcessors; private final int maxChannelsPerProcessor; private final int numThreads; + private final boolean isDurableStorage; private StorageAdapter adapter; private RowSignature signature; @@ -195,7 +196,8 @@ public ParameterizedCasesTest( int numChannels, int maxActiveProcessors, int maxChannelsPerProcessor, - int numThreads + int numThreads, + boolean isDurableStorage ) { this.maxRowsPerFrame = maxRowsPerFrame; @@ -204,6 +206,7 @@ public ParameterizedCasesTest( this.maxActiveProcessors = maxActiveProcessors; this.maxChannelsPerProcessor = maxChannelsPerProcessor; this.numThreads = numThreads; + this.isDurableStorage = isDurableStorage; } @Parameterized.Parameters( @@ -212,7 +215,8 @@ public ParameterizedCasesTest( + "numChannels = {2}, " + "maxActiveProcessors = {3}, " + "maxChannelsPerProcessor = {4}, " - + "numThreads = {5}" + + "numThreads = {5}, " + + "isDurableStorage = {6}" ) public static Iterable constructorFeeder() { @@ -224,17 +228,20 @@ public static Iterable constructorFeeder() for (int maxActiveProcessors : new int[]{1, 2, 4}) { for (int maxChannelsPerProcessor : new int[]{2, 3, 8}) { for (int numThreads : new int[]{1, 3}) { - if (maxActiveProcessors >= maxChannelsPerProcessor) { - constructors.add( - new Object[]{ - maxRowsPerFrame, - maxBytesPerFrame, - numChannels, - maxActiveProcessors, - maxChannelsPerProcessor, - numThreads - } - ); + for (boolean isDurableStorage : new boolean[]{true, false}) { + if (maxActiveProcessors >= maxChannelsPerProcessor) { + constructors.add( + new Object[]{ + maxRowsPerFrame, + maxBytesPerFrame, + numChannels, + maxActiveProcessors, + maxChannelsPerProcessor, + numThreads, + isDurableStorage + } + ); + } } } } @@ -292,16 +299,24 @@ private void setUpInputChannels(final ClusterBy clusterBy) throws Exception private OutputChannels verifySuperSorter( final ClusterBy clusterBy, - final ClusterByPartitions clusterByPartitions, - boolean isDurableStorage + final ClusterByPartitions clusterByPartitions ) throws Exception { + final File tempFolder = temporaryFolder.newFolder(); + final OutputChannelFactory outputChannelFactory = isDurableStorage ? new DurableStorageOutputChannelFactory( + "0", + 0, + 0, + "0", + maxBytesPerFrame, + new LocalFileStorageConnector(tempFolder), + tempFolder + ) : new FileOutputChannelFactory(tempFolder, maxBytesPerFrame); final RowKeyReader keyReader = clusterBy.keyReader(signature); final Comparator keyComparator = clusterBy.keyComparator(); final SettableFuture clusterByPartitionsFuture = SettableFuture.create(); final SuperSorterProgressTracker superSorterProgressTracker = new SuperSorterProgressTracker(); - File tempFolder = temporaryFolder.newFolder(); final SuperSorter superSorter = new SuperSorter( inputChannels, frameReader, @@ -309,15 +324,7 @@ private OutputChannels verifySuperSorter( clusterByPartitionsFuture, exec, new FileOutputChannelFactory(tempFolder, maxBytesPerFrame), - isDurableStorage ? new DurableStorageOutputChannelFactory( - "0", - 0, - 0, - "0", - maxBytesPerFrame, - new LocalFileStorageConnector(tempFolder), - tempFolder - ) : new FileOutputChannelFactory(tempFolder, maxBytesPerFrame), + outputChannelFactory, maxActiveProcessors, maxChannelsPerProcessor, -1, @@ -417,7 +424,7 @@ public void test_clusterByQualityLongAscRowNumberAsc_onePartition() throws Excep ); setUpInputChannels(clusterBy); - verifySuperSorter(clusterBy, ClusterByPartitions.oneUniversalPartition(), false); + verifySuperSorter(clusterBy, ClusterByPartitions.oneUniversalPartition()); } @Test @@ -441,8 +448,7 @@ public void test_clusterByQualityLongAscRowNumberAsc_twoPartitionsOneEmpty() thr new ClusterByPartition(null, zeroZero), // empty partition new ClusterByPartition(zeroZero, null) // all data goes in here ) - ), - false + ) ); // Verify that one of the partitions is actually empty. @@ -504,7 +510,7 @@ public void test_clusterByQualityDescRowNumberAsc_fourPartitions() throws Except Assert.assertEquals(4, partitions.size()); - verifySuperSorter(clusterBy, partitions, false); + verifySuperSorter(clusterBy, partitions); } @Test @@ -544,7 +550,7 @@ public void test_clusterByTimeAscMarketAscRowNumberAsc_fourPartitions() throws E Assert.assertEquals(4, partitions.size()); - verifySuperSorter(clusterBy, partitions, false); + verifySuperSorter(clusterBy, partitions); } @Test @@ -583,7 +589,7 @@ public void test_clusterByPlacementishDescRowNumberAsc_fourPartitions() throws E Assert.assertEquals(4, partitions.size()); - verifySuperSorter(clusterBy, partitions, false); + verifySuperSorter(clusterBy, partitions); } @Test @@ -622,7 +628,7 @@ public void test_clusterByQualityLongDescRowNumberAsc_fourPartitions() throws Ex Assert.assertEquals(4, partitions.size()); - verifySuperSorter(clusterBy, partitions, false); + verifySuperSorter(clusterBy, partitions); } @Test @@ -661,7 +667,7 @@ public void test_clusterByQualityLongDescRowNumberAsc_fourPartitions_durableStor Assert.assertEquals(4, partitions.size()); - verifySuperSorter(clusterBy, partitions, true); + verifySuperSorter(clusterBy, partitions); } private RowKey createKey(final ClusterBy clusterBy, final Object... objects) From b052196bc2c848e576bce3597a080e1206acd09a Mon Sep 17 00:00:00 2001 From: Rohan Garg Date: Tue, 22 Nov 2022 23:35:22 +0530 Subject: [PATCH 05/15] Review fixes --- .../storage/local/LocalFileStorageConnector.java | 14 ++++++++++++++ .../org/apache/druid/msq/exec/ControllerImpl.java | 3 +-- .../msq/indexing/CountingOutputChannelFactory.java | 4 ++-- .../storage/s3/output/S3StorageConnector.java | 9 +++++++++ .../DurableStorageOutputChannelFactory.java | 4 +++- ...ableStoragePartitionedReadableFrameChannel.java | 2 +- .../channel/PartitionedReadableFrameChannel.java | 10 +++++++++- .../BlockingQueueOutputChannelFactory.java | 2 +- .../frame/processor/FileOutputChannelFactory.java | 4 ++-- .../frame/processor/OutputChannelFactory.java | 2 +- .../apache/druid/frame/processor/SuperSorter.java | 6 +++--- 11 files changed, 46 insertions(+), 14 deletions(-) diff --git a/core/src/main/java/org/apache/druid/storage/local/LocalFileStorageConnector.java b/core/src/main/java/org/apache/druid/storage/local/LocalFileStorageConnector.java index 4456eced9039..366167dcdc94 100644 --- a/core/src/main/java/org/apache/druid/storage/local/LocalFileStorageConnector.java +++ b/core/src/main/java/org/apache/druid/storage/local/LocalFileStorageConnector.java @@ -27,6 +27,7 @@ import org.apache.druid.storage.StorageConnector; import java.io.File; +import java.io.FileNotFoundException; import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; @@ -68,6 +69,19 @@ public InputStream read(String path) throws IOException @Override public InputStream readRange(String path, long from, long size) throws IOException { + if (!pathExists(path)) { + throw new FileNotFoundException("Unable to find file " + fileWithBasePath(path).toPath() + " for reading"); + } + long length = fileWithBasePath(path).length(); + if (from < 0 || size < 0 || (from + size) > length) { + throw new IAE( + "Invalid arguments for reading %s. from = %d, readSize = %d, fileSize = %d", + fileWithBasePath(path).toPath(), + from, + size, + length + ); + } FileChannel fileChannel = FileChannel.open(fileWithBasePath(path).toPath(), StandardOpenOption.READ); return new BoundedInputStream(Channels.newInputStream(fileChannel.position(from)), size); } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java index feb72c5a7bd9..75bcda8efc93 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java @@ -156,7 +156,6 @@ import org.apache.druid.msq.util.MultiStageQueryContext; import org.apache.druid.msq.util.PassthroughAggregatorFactory; import org.apache.druid.query.Query; -import org.apache.druid.query.QueryContext; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.groupby.GroupByQuery; import org.apache.druid.query.groupby.GroupByQueryConfig; @@ -279,7 +278,7 @@ public ControllerImpl( this.task = task; this.context = context; this.isDurableStageStorageEnabled = MultiStageQueryContext.isDurableShuffleStorageEnabled( - QueryContext.of(task.getContext()) + task.getQuerySpec().getQuery().context() ); } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/CountingOutputChannelFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/CountingOutputChannelFactory.java index 7c905e589a6a..4197ba7e661b 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/CountingOutputChannelFactory.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/CountingOutputChannelFactory.java @@ -57,9 +57,9 @@ public OutputChannel openChannel(int partitionNumber) throws IOException } @Override - public PartitionedOutputChannel openChannel(String name, boolean deleteAfterRead) throws IOException + public PartitionedOutputChannel openPartitionedChannel(String name, boolean deleteAfterRead) throws IOException { - final PartitionedOutputChannel baseChannel = baseFactory.openChannel(name, deleteAfterRead); + final PartitionedOutputChannel baseChannel = baseFactory.openPartitionedChannel(name, deleteAfterRead); return baseChannel.mapWritableChannel( baseWritableChannel -> diff --git a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3StorageConnector.java b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3StorageConnector.java index aaeec35bc5a3..d81624d89359 100644 --- a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3StorageConnector.java +++ b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3StorageConnector.java @@ -27,6 +27,7 @@ import com.google.common.base.Joiner; import org.apache.druid.data.input.impl.RetryingInputStream; import org.apache.druid.data.input.impl.prefetch.ObjectOpenFunction; +import org.apache.druid.java.util.common.IAE; import org.apache.druid.storage.StorageConnector; import org.apache.druid.storage.s3.S3Utils; import org.apache.druid.storage.s3.ServerSideEncryptingAmazonS3; @@ -92,6 +93,14 @@ public InputStream open(GetObjectRequest object, long offset) @Override public InputStream readRange(String path, long from, long size) throws IOException { + if (from < 0 || size < 0) { + throw new IAE( + "Invalid arguments for reading %s. from = %d, readSize = %d", + objectPath(path), + from, + size + ); + } return new RetryingInputStream<>( new GetObjectRequest(config.getBucket(), objectPath(path)).withRange(from, from + size - 1), new ObjectOpenFunction() diff --git a/processing/src/main/java/org/apache/druid/frame/channel/DurableStorageOutputChannelFactory.java b/processing/src/main/java/org/apache/druid/frame/channel/DurableStorageOutputChannelFactory.java index dddf451c85c3..470475f23a17 100644 --- a/processing/src/main/java/org/apache/druid/frame/channel/DurableStorageOutputChannelFactory.java +++ b/processing/src/main/java/org/apache/druid/frame/channel/DurableStorageOutputChannelFactory.java @@ -163,7 +163,7 @@ public OutputChannel openChannel(int partitionNumber) throws IOException } @Override - public PartitionedOutputChannel openChannel(String name, boolean deleteAfterRead) throws IOException + public PartitionedOutputChannel openPartitionedChannel(String name, boolean deleteAfterRead) throws IOException { final String fileName = DurableStorageUtils.getOutputsFileNameForPath( controllerTaskId, @@ -205,6 +205,8 @@ public PartitionedOutputChannel openChannel(String name, boolean deleteAfterRead int footerLength = trailer.getInt(Integer.BYTES * 2L); // read the footer into a file and map it to memory + FileUtils.mkdirp(footerFile.getParentFile()); + Preconditions.checkState(footerFile.createNewFile(), "Unable to create local footer file"); try (FileOutputStream footerFileStream = new FileOutputStream(footerFile); InputStream footerInputStream = storageConnector.readRange(fileName, channelSize - footerLength, footerLength)) { diff --git a/processing/src/main/java/org/apache/druid/frame/channel/DurableStoragePartitionedReadableFrameChannel.java b/processing/src/main/java/org/apache/druid/frame/channel/DurableStoragePartitionedReadableFrameChannel.java index 3b3446f61217..0e0e6680df37 100644 --- a/processing/src/main/java/org/apache/druid/frame/channel/DurableStoragePartitionedReadableFrameChannel.java +++ b/processing/src/main/java/org/apache/druid/frame/channel/DurableStoragePartitionedReadableFrameChannel.java @@ -53,7 +53,7 @@ public DurableStoragePartitionedReadableFrameChannel( } @Override - public ReadableFrameChannel openChannel(int partitionNumber) + public ReadableFrameChannel getReadableFrameChannel(int partitionNumber) { FrameFileFooter frameFileFooter = frameFileFooterSupplier.get(); // find the range to read for partition diff --git a/processing/src/main/java/org/apache/druid/frame/channel/PartitionedReadableFrameChannel.java b/processing/src/main/java/org/apache/druid/frame/channel/PartitionedReadableFrameChannel.java index 267bb76c9fe4..4edd904afda3 100644 --- a/processing/src/main/java/org/apache/druid/frame/channel/PartitionedReadableFrameChannel.java +++ b/processing/src/main/java/org/apache/druid/frame/channel/PartitionedReadableFrameChannel.java @@ -20,6 +20,7 @@ package org.apache.druid.frame.channel; import java.io.Closeable; +import java.io.IOException; /** * Provides an interface to read a partitioned frame channel. The channel might have frames with multiple partitions @@ -32,5 +33,12 @@ public interface PartitionedReadableFrameChannel extends Closeable * @param partitionNumber the partition to read * @return a ReadableFrameChannel for the partition queried */ - ReadableFrameChannel openChannel(int partitionNumber); + ReadableFrameChannel getReadableFrameChannel(int partitionNumber); + + /** + * Releases any resources associated with this readable channel. After calling this, you should not call any other + * methods on the channel. + */ + @Override + void close() throws IOException; } diff --git a/processing/src/main/java/org/apache/druid/frame/processor/BlockingQueueOutputChannelFactory.java b/processing/src/main/java/org/apache/druid/frame/processor/BlockingQueueOutputChannelFactory.java index b7e3b4706ad7..20426b346bfb 100644 --- a/processing/src/main/java/org/apache/druid/frame/processor/BlockingQueueOutputChannelFactory.java +++ b/processing/src/main/java/org/apache/druid/frame/processor/BlockingQueueOutputChannelFactory.java @@ -47,7 +47,7 @@ public OutputChannel openChannel(final int partitionNumber) } @Override - public PartitionedOutputChannel openChannel(String name, boolean deleteAfterRead) + public PartitionedOutputChannel openPartitionedChannel(String name, boolean deleteAfterRead) { throw new UnsupportedOperationException("Opening in-memory partitioned channels is not supported"); } diff --git a/processing/src/main/java/org/apache/druid/frame/processor/FileOutputChannelFactory.java b/processing/src/main/java/org/apache/druid/frame/processor/FileOutputChannelFactory.java index dc1417458cb4..34e01f63d884 100644 --- a/processing/src/main/java/org/apache/druid/frame/processor/FileOutputChannelFactory.java +++ b/processing/src/main/java/org/apache/druid/frame/processor/FileOutputChannelFactory.java @@ -95,7 +95,7 @@ public OutputChannel openChannel(int partitionNumber) throws IOException } @Override - public PartitionedOutputChannel openChannel(String name, boolean deleteAfterRead) throws IOException + public PartitionedOutputChannel openPartitionedChannel(String name, boolean deleteAfterRead) throws IOException { FileUtils.mkdirp(fileChannelsDirectory); final File file = new File(fileChannelsDirectory, name); @@ -124,7 +124,7 @@ public PartitionedOutputChannel openChannel(String name, boolean deleteAfterRead () -> new PartitionedReadableFrameChannel() { @Override - public ReadableFrameChannel openChannel(int partitionNumber) + public ReadableFrameChannel getReadableFrameChannel(int partitionNumber) { FrameFile fileHandle = frameFileSupplier.get(); fileHandle = fileHandle.newReference(); diff --git a/processing/src/main/java/org/apache/druid/frame/processor/OutputChannelFactory.java b/processing/src/main/java/org/apache/druid/frame/processor/OutputChannelFactory.java index 4b0ca79aad8f..5568f7494665 100644 --- a/processing/src/main/java/org/apache/druid/frame/processor/OutputChannelFactory.java +++ b/processing/src/main/java/org/apache/druid/frame/processor/OutputChannelFactory.java @@ -34,7 +34,7 @@ public interface OutputChannelFactory /** * Create a channel pair tagged with a particular name and a flag to delete the channel data after its read. */ - PartitionedOutputChannel openChannel(String name, boolean deleteAfterRead) throws IOException; + PartitionedOutputChannel openPartitionedChannel(String name, boolean deleteAfterRead) throws IOException; /** * Create a non-writable, always-empty channel pair tagged with a particular partition number. diff --git a/processing/src/main/java/org/apache/druid/frame/processor/SuperSorter.java b/processing/src/main/java/org/apache/druid/frame/processor/SuperSorter.java index b4bb2c37ed5f..0be8c8970f26 100644 --- a/processing/src/main/java/org/apache/druid/frame/processor/SuperSorter.java +++ b/processing/src/main/java/org/apache/druid/frame/processor/SuperSorter.java @@ -512,7 +512,7 @@ private boolean runNextMiddleMerger() levelAndRankToReadableChannelMap.remove(levelAndRankKey) .getReadableChannelSupplier() .get(); - in.add(partitionedReadableFrameChannel.openChannel(0)); + in.add(partitionedReadableFrameChannel.getReadableFrameChannel(0)); partitionedReadableFrameChannels.add(partitionedReadableFrameChannel); } } @@ -569,7 +569,7 @@ private boolean runNextUltimateMerger() levelAndRankToReadableChannelMap.get(StringUtils.format("merged.%d.%d", inLevel, i)) .getReadableChannelSupplier() .get() - .openChannel(ultimateMergersRunSoFar) + .getReadableFrameChannel(ultimateMergersRunSoFar) ); } @@ -603,7 +603,7 @@ private void runMerger( writableChannel = outputChannel.getWritableChannel(); frameAllocator = outputChannel.getFrameMemoryAllocator(); } else { - PartitionedOutputChannel partitionedOutputChannel = intermediateOutputChannelFactory.openChannel( + PartitionedOutputChannel partitionedOutputChannel = intermediateOutputChannelFactory.openPartitionedChannel( levelAndRankKey, true ); From 159c5d7544d6c32c8dd60bb6bd53f68bac607b2b Mon Sep 17 00:00:00 2001 From: Rohan Garg Date: Mon, 28 Nov 2022 01:57:05 +0530 Subject: [PATCH 06/15] Move durable storage classes --- .../java/org/apache/druid/msq/exec/ControllerImpl.java | 2 +- .../main/java/org/apache/druid/msq/exec/WorkerImpl.java | 4 ++-- .../apache/druid/msq/indexing/DurableStorageCleaner.java | 2 +- .../msq/shuffle/DurableStorageInputChannelFactory.java | 5 +++-- .../java/org/apache/druid/msq/exec/MSQSelectTest.java | 2 +- .../druid/msq/indexing/DurableStorageCleanerTest.java | 2 +- .../DurableStorageOutputChannelFactory.java | 9 +++++---- .../frame/{channel => util}/DurableStorageUtils.java | 2 +- .../apache/druid/frame/processor/SuperSorterTest.java | 1 - 9 files changed, 15 insertions(+), 14 deletions(-) rename processing/src/main/java/org/apache/druid/frame/{channel => processor}/DurableStorageOutputChannelFactory.java (97%) rename processing/src/main/java/org/apache/druid/frame/{channel => util}/DurableStorageUtils.java (98%) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java index 9bf71e08f3e3..6856c17419b1 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java @@ -40,7 +40,6 @@ import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.frame.allocation.ArenaMemoryAllocator; -import org.apache.druid.frame.channel.DurableStorageUtils; import org.apache.druid.frame.channel.FrameChannelSequence; import org.apache.druid.frame.key.ClusterBy; import org.apache.druid.frame.key.ClusterByPartition; @@ -50,6 +49,7 @@ import org.apache.druid.frame.key.SortColumn; import org.apache.druid.frame.processor.FrameProcessorExecutor; import org.apache.druid.frame.processor.FrameProcessors; +import org.apache.druid.frame.util.DurableStorageUtils; import org.apache.druid.indexer.TaskState; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.common.LockGranularity; diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java index 9b15927bdb8f..93d57a815685 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java @@ -32,8 +32,6 @@ import it.unimi.dsi.fastutil.bytes.ByteArrays; import org.apache.druid.frame.allocation.ArenaMemoryAllocator; import org.apache.druid.frame.channel.BlockingQueueFrameChannel; -import org.apache.druid.frame.channel.DurableStorageOutputChannelFactory; -import org.apache.druid.frame.channel.DurableStorageUtils; import org.apache.druid.frame.channel.ReadableFileFrameChannel; import org.apache.druid.frame.channel.ReadableFrameChannel; import org.apache.druid.frame.channel.ReadableNilFrameChannel; @@ -43,6 +41,7 @@ import org.apache.druid.frame.key.ClusterByPartitions; import org.apache.druid.frame.processor.BlockingQueueOutputChannelFactory; import org.apache.druid.frame.processor.Bouncer; +import org.apache.druid.frame.processor.DurableStorageOutputChannelFactory; import org.apache.druid.frame.processor.FileOutputChannelFactory; import org.apache.druid.frame.processor.FrameChannelMuxer; import org.apache.druid.frame.processor.FrameProcessor; @@ -52,6 +51,7 @@ import org.apache.druid.frame.processor.OutputChannels; import org.apache.druid.frame.processor.SuperSorter; import org.apache.druid.frame.processor.SuperSorterProgressTracker; +import org.apache.druid.frame.util.DurableStorageUtils; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.java.util.common.FileUtils; import org.apache.druid.java.util.common.ISE; diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/DurableStorageCleaner.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/DurableStorageCleaner.java index b08ec2de146c..b89af761123d 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/DurableStorageCleaner.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/DurableStorageCleaner.java @@ -24,7 +24,7 @@ import com.google.common.collect.Sets; import com.google.inject.Inject; import com.google.inject.Provider; -import org.apache.druid.frame.channel.DurableStorageUtils; +import org.apache.druid.frame.util.DurableStorageUtils; import org.apache.druid.indexing.overlord.TaskMaster; import org.apache.druid.indexing.overlord.TaskRunner; import org.apache.druid.indexing.overlord.TaskRunnerWorkItem; diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/DurableStorageInputChannelFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/DurableStorageInputChannelFactory.java index 17eadcedb7dc..bc69cdd4b734 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/DurableStorageInputChannelFactory.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/DurableStorageInputChannelFactory.java @@ -21,9 +21,10 @@ import com.google.common.base.Preconditions; import org.apache.commons.io.IOUtils; -import org.apache.druid.frame.channel.DurableStorageUtils; import org.apache.druid.frame.channel.ReadableFrameChannel; import org.apache.druid.frame.channel.ReadableInputStreamFrameChannel; +import org.apache.druid.frame.processor.DurableStorageOutputChannelFactory; +import org.apache.druid.frame.util.DurableStorageUtils; import org.apache.druid.java.util.common.IOE; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.RetryUtils; @@ -133,7 +134,7 @@ public ReadableFrameChannel openChannel(StageId stageId, int workerNumber, int p * 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 org.apache.druid.frame.channel.DurableStorageOutputChannelFactory#createSuccessFile(String)} + * More information at {@link DurableStorageOutputChannelFactory#createSuccessFile(String)} */ public String findSuccessfulPartitionOutput( final String controllerTaskId, diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java index 717b6ae6b479..f0ca8f92a8cd 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java @@ -24,7 +24,7 @@ import org.apache.druid.common.config.NullHandling; import org.apache.druid.data.input.impl.JsonInputFormat; import org.apache.druid.data.input.impl.LocalInputSource; -import org.apache.druid.frame.channel.DurableStorageUtils; +import org.apache.druid.frame.util.DurableStorageUtils; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.math.expr.ExprMacroTable; diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/DurableStorageCleanerTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/DurableStorageCleanerTest.java index 948e91815250..664bd8f9a2b1 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/DurableStorageCleanerTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/DurableStorageCleanerTest.java @@ -21,7 +21,7 @@ import com.google.common.base.Optional; import com.google.common.collect.ImmutableList; -import org.apache.druid.frame.channel.DurableStorageUtils; +import org.apache.druid.frame.util.DurableStorageUtils; import org.apache.druid.indexer.TaskLocation; import org.apache.druid.indexing.overlord.TaskMaster; import org.apache.druid.indexing.overlord.TaskRunner; diff --git a/processing/src/main/java/org/apache/druid/frame/channel/DurableStorageOutputChannelFactory.java b/processing/src/main/java/org/apache/druid/frame/processor/DurableStorageOutputChannelFactory.java similarity index 97% rename from processing/src/main/java/org/apache/druid/frame/channel/DurableStorageOutputChannelFactory.java rename to processing/src/main/java/org/apache/druid/frame/processor/DurableStorageOutputChannelFactory.java index 470475f23a17..27ecab62e45f 100644 --- a/processing/src/main/java/org/apache/druid/frame/channel/DurableStorageOutputChannelFactory.java +++ b/processing/src/main/java/org/apache/druid/frame/processor/DurableStorageOutputChannelFactory.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.frame.channel; +package org.apache.druid.frame.processor; import com.google.common.base.Preconditions; import com.google.common.base.Suppliers; @@ -26,11 +26,12 @@ import org.apache.datasketches.memory.Memory; import org.apache.druid.frame.Frame; import org.apache.druid.frame.allocation.ArenaMemoryAllocator; +import org.apache.druid.frame.channel.DurableStoragePartitionedReadableFrameChannel; +import org.apache.druid.frame.channel.ReadableInputStreamFrameChannel; +import org.apache.druid.frame.channel.WritableFrameFileChannel; import org.apache.druid.frame.file.FrameFileFooter; 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.frame.processor.PartitionedOutputChannel; +import org.apache.druid.frame.util.DurableStorageUtils; import org.apache.druid.java.util.common.FileUtils; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.MappedByteBufferHandler; diff --git a/processing/src/main/java/org/apache/druid/frame/channel/DurableStorageUtils.java b/processing/src/main/java/org/apache/druid/frame/util/DurableStorageUtils.java similarity index 98% rename from processing/src/main/java/org/apache/druid/frame/channel/DurableStorageUtils.java rename to processing/src/main/java/org/apache/druid/frame/util/DurableStorageUtils.java index 374d4e6820f2..7104806fedda 100644 --- a/processing/src/main/java/org/apache/druid/frame/channel/DurableStorageUtils.java +++ b/processing/src/main/java/org/apache/druid/frame/util/DurableStorageUtils.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.frame.channel; +package org.apache.druid.frame.util; import org.apache.druid.common.utils.IdUtils; import org.apache.druid.java.util.common.StringUtils; diff --git a/processing/src/test/java/org/apache/druid/frame/processor/SuperSorterTest.java b/processing/src/test/java/org/apache/druid/frame/processor/SuperSorterTest.java index 553d0c340711..ff8e65be29f2 100644 --- a/processing/src/test/java/org/apache/druid/frame/processor/SuperSorterTest.java +++ b/processing/src/test/java/org/apache/druid/frame/processor/SuperSorterTest.java @@ -27,7 +27,6 @@ import org.apache.druid.frame.FrameType; import org.apache.druid.frame.allocation.ArenaMemoryAllocator; import org.apache.druid.frame.channel.BlockingQueueFrameChannel; -import org.apache.druid.frame.channel.DurableStorageOutputChannelFactory; import org.apache.druid.frame.channel.ReadableFileFrameChannel; import org.apache.druid.frame.channel.ReadableFrameChannel; import org.apache.druid.frame.channel.WritableFrameChannel; From 98c91c700a1d09456cd85a8319976a44b7b4b8c4 Mon Sep 17 00:00:00 2001 From: Rohan Garg Date: Wed, 30 Nov 2022 12:29:29 +0530 Subject: [PATCH 07/15] Review fixes --- .../local/LocalFileStorageConnectorTest.java | 4 ++-- .../s3/output/S3StorageConnectorTest.java | 4 ++-- .../druid/frame/processor/SuperSorter.java | 19 ++++++++++++++----- 3 files changed, 18 insertions(+), 9 deletions(-) diff --git a/core/src/test/java/org/apache/druid/storage/local/LocalFileStorageConnectorTest.java b/core/src/test/java/org/apache/druid/storage/local/LocalFileStorageConnectorTest.java index 4fd24c672603..d1eaf3a73b11 100644 --- a/core/src/test/java/org/apache/druid/storage/local/LocalFileStorageConnectorTest.java +++ b/core/src/test/java/org/apache/druid/storage/local/LocalFileStorageConnectorTest.java @@ -161,7 +161,7 @@ public void testReadRange() throws Exception outputStream.write(data.getBytes(StandardCharsets.UTF_8)); } - // normal reads + // non empty reads for (int start = 0; start < data.length(); start++) { for (int length = 1; length <= data.length() - start; length++) { InputStream is = storageConnector.readRange(uuid, start, length); @@ -172,7 +172,7 @@ public void testReadRange() throws Exception } } - // 0 size read + // empty read InputStream is = storageConnector.readRange(uuid, 0, 0); byte[] dataBytes = new byte[0]; Assert.assertEquals(is.read(dataBytes), -1); diff --git a/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/S3StorageConnectorTest.java b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/S3StorageConnectorTest.java index a7a5b6095eef..154918cc2d93 100644 --- a/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/S3StorageConnectorTest.java +++ b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/S3StorageConnectorTest.java @@ -123,7 +123,7 @@ public void testReadRange() throws IOException { String data = "test"; - // normal reads + // non empty reads for (int start = 0; start < data.length(); start++) { for (int length = 1; length <= data.length() - start; length++) { String dataQueried = data.substring(start, start + length); @@ -148,7 +148,7 @@ public void testReadRange() throws IOException } } - // 0 size read + // empty read EasyMock.reset(S3_CLIENT); S3Object s3Object = new S3Object(); s3Object.setObjectContent( diff --git a/processing/src/main/java/org/apache/druid/frame/processor/SuperSorter.java b/processing/src/main/java/org/apache/druid/frame/processor/SuperSorter.java index 0be8c8970f26..89653b9dd745 100644 --- a/processing/src/main/java/org/apache/druid/frame/processor/SuperSorter.java +++ b/processing/src/main/java/org/apache/druid/frame/processor/SuperSorter.java @@ -507,7 +507,7 @@ private boolean runNextMiddleMerger() final List partitionedReadableFrameChannels = new ArrayList<>(); for (long i = currentSetStart; i < currentSetStart + maxChannelsPerProcessor; i++) { if (inputsReady.remove(i)) { - String levelAndRankKey = StringUtils.format("merged.%d.%d", inLevel, i); + String levelAndRankKey = mergerOutputFileName(inLevel, i); PartitionedReadableFrameChannel partitionedReadableFrameChannel = levelAndRankToReadableChannelMap.remove(levelAndRankKey) .getReadableChannelSupplier() @@ -566,7 +566,7 @@ private boolean runNextUltimateMerger() for (long i = 0; i < numInputs; i++) { in.add( - levelAndRankToReadableChannelMap.get(StringUtils.format("merged.%d.%d", inLevel, i)) + levelAndRankToReadableChannelMap.get(mergerOutputFileName(inLevel, i)) .getReadableChannelSupplier() .get() .getReadableFrameChannel(ultimateMergersRunSoFar) @@ -588,13 +588,13 @@ private void runMerger( final long rank, final List in, @Nullable final ClusterByPartitions partitions, - List partitionedReadableChannelsToClose + final List partitionedReadableChannelsToClose ) { try { final WritableFrameChannel writableChannel; final MemoryAllocator frameAllocator; - String levelAndRankKey = StringUtils.format("merged.%d.%d", level, rank); + String levelAndRankKey = mergerOutputFileName(level, rank); if (totalMergingLevels != UNKNOWN_LEVEL && level == totalMergingLevels - 1) { final int intRank = Ints.checkedCast(rank); @@ -639,7 +639,10 @@ private void runMerger( partitionedReadableFrameChannel.close(); } catch (IOException e) { - log.warn(e, "Could not close channel for level [%d] and rank [%d]", level, rank); + throw new UncheckedIOException( + StringUtils.format("Could not close channel for level [%d] and rank [%d]", level, rank), + e + ); } } } @@ -822,6 +825,12 @@ private void cleanUp() inputChannelsToRead.clear(); } + private String mergerOutputFileName(final int level, final long rank) + { + return StringUtils.format("merged.%d.%d", level, rank); + } + + /** * Returns a string encapsulating the current state of this object. */ From 7b41c0b8821d2d0989bc07059a90449b603b6398 Mon Sep 17 00:00:00 2001 From: Rohan Garg Date: Mon, 9 Jan 2023 00:53:27 +0530 Subject: [PATCH 08/15] Review comments --- .../storage/s3/output/S3StorageConnector.java | 32 ++++--------------- .../druid/frame/file/FrameFileFooter.java | 1 + .../DurableStorageOutputChannelFactory.java | 2 +- .../processor/PartitionedOutputChannel.java | 5 +++ .../druid/frame/util/DurableStorageUtils.java | 5 ++- 5 files changed, 18 insertions(+), 27 deletions(-) diff --git a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3StorageConnector.java b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3StorageConnector.java index d81624d89359..184e1bdfe100 100644 --- a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3StorageConnector.java +++ b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3StorageConnector.java @@ -64,30 +64,7 @@ public boolean pathExists(String path) @Override public InputStream read(String path) throws IOException { - return new RetryingInputStream<>( - new GetObjectRequest(config.getBucket(), objectPath(path)), - new ObjectOpenFunction() - { - @Override - public InputStream open(GetObjectRequest object) - { - return s3Client.getObject(object).getObjectContent(); - } - - @Override - public InputStream open(GetObjectRequest object, long offset) - { - final GetObjectRequest offsetObjectRequest = new GetObjectRequest( - object.getBucketName(), - object.getKey() - ); - offsetObjectRequest.setRange(offset); - return open(offsetObjectRequest); - } - }, - S3Utils.S3RETRY, - config.getMaxRetry() - ); + return buildInputStream(new GetObjectRequest(config.getBucket(), objectPath(path))); } @Override @@ -101,8 +78,13 @@ public InputStream readRange(String path, long from, long size) throws IOExcepti size ); } + return buildInputStream(new GetObjectRequest(config.getBucket(), objectPath(path)).withRange(from, from + size - 1)); + } + + private RetryingInputStream buildInputStream(GetObjectRequest getObjectRequest) throws IOException + { return new RetryingInputStream<>( - new GetObjectRequest(config.getBucket(), objectPath(path)).withRange(from, from + size - 1), + getObjectRequest, new ObjectOpenFunction() { @Override diff --git a/processing/src/main/java/org/apache/druid/frame/file/FrameFileFooter.java b/processing/src/main/java/org/apache/druid/frame/file/FrameFileFooter.java index e460c1e57084..3d2bd29f3423 100644 --- a/processing/src/main/java/org/apache/druid/frame/file/FrameFileFooter.java +++ b/processing/src/main/java/org/apache/druid/frame/file/FrameFileFooter.java @@ -29,6 +29,7 @@ /** * Encapsulation for Æ’rame file footer related operations. The footer must be wrapped in a memory object (the memory * can be physical or mmaped). Some verifications are also done on the footer to see if it is not corrupted. + * The schema for footer is as described by {@link FrameFile}. */ public class FrameFileFooter { diff --git a/processing/src/main/java/org/apache/druid/frame/processor/DurableStorageOutputChannelFactory.java b/processing/src/main/java/org/apache/druid/frame/processor/DurableStorageOutputChannelFactory.java index 27ecab62e45f..2ef32b666af3 100644 --- a/processing/src/main/java/org/apache/druid/frame/processor/DurableStorageOutputChannelFactory.java +++ b/processing/src/main/java/org/apache/druid/frame/processor/DurableStorageOutputChannelFactory.java @@ -185,7 +185,7 @@ public PartitionedOutputChannel openPartitionedChannel(String name, boolean dele final Supplier channelSizeSupplier = countingOutputStream::getCount; final File footerFile = new File(tmpDir, fileName + "_footer"); - // build supplier for reader the footer of the underlying frame file + // build supplier for reading the footer of the underlying frame file final Supplier frameFileFooterSupplier = Suppliers.memoize(() -> { try { // read trailer and find the footer size diff --git a/processing/src/main/java/org/apache/druid/frame/processor/PartitionedOutputChannel.java b/processing/src/main/java/org/apache/druid/frame/processor/PartitionedOutputChannel.java index 6ea9d2909be4..3e455545b049 100644 --- a/processing/src/main/java/org/apache/druid/frame/processor/PartitionedOutputChannel.java +++ b/processing/src/main/java/org/apache/druid/frame/processor/PartitionedOutputChannel.java @@ -30,6 +30,11 @@ import java.util.function.Function; import java.util.function.Supplier; +/** + * A channel which can contain multiple partitions of data. It is used by {@link SuperSorter} currently to write multiple + * partitions on the same channel. The readable channel provided to the caller is an instance of + * {@link PartitionedReadableFrameChannel} which allows the caller to open a readable channel for the desired partition. + */ public class PartitionedOutputChannel { @Nullable diff --git a/processing/src/main/java/org/apache/druid/frame/util/DurableStorageUtils.java b/processing/src/main/java/org/apache/druid/frame/util/DurableStorageUtils.java index 7104806fedda..6e6bbd9fe927 100644 --- a/processing/src/main/java/org/apache/druid/frame/util/DurableStorageUtils.java +++ b/processing/src/main/java/org/apache/druid/frame/util/DurableStorageUtils.java @@ -103,7 +103,10 @@ public static String getPartitionOutputsFileNameForPartition( ); } - + /** + * Fetches the file location where a particular worker writes the data corresponding to a particular stage + * and a custom path name + */ public static String getOutputsFileNameForPath( final String controllerTaskId, final int stageNumber, From 5855d698615e724879bb486c0297e66331f2a7b4 Mon Sep 17 00:00:00 2001 From: Rohan Garg Date: Mon, 28 Nov 2022 01:41:44 +0530 Subject: [PATCH 09/15] Add composed output channels --- .../org/apache/druid/msq/exec/WorkerImpl.java | 4 +- .../msq/querykit/BroadcastJoinHelperTest.java | 6 +- .../channel/BoundedWritableFrameChannel.java | 62 +++++++ .../druid/frame/channel/ByteTracker.java | 55 ++++++ .../ComposingReadableFrameChannel.java | 143 +++++++++++++++ .../ComposingWritableFrameChannel.java | 95 ++++++++++ .../apache/druid/frame/file/FrameFile.java | 27 ++- .../ComposingOutputChannelFactory.java | 163 ++++++++++++++++++ .../processor/FileOutputChannelFactory.java | 64 ++++--- .../druid/frame/processor/OutputChannel.java | 5 + .../druid/frame/file/FrameFileTest.java | 24 +-- .../druid/frame/file/FrameFileWriterTest.java | 2 +- .../processor/FrameProcessorExecutorTest.java | 4 +- .../frame/processor/SuperSorterTest.java | 45 +++-- 14 files changed, 629 insertions(+), 70 deletions(-) create mode 100644 processing/src/main/java/org/apache/druid/frame/channel/BoundedWritableFrameChannel.java create mode 100644 processing/src/main/java/org/apache/druid/frame/channel/ByteTracker.java create mode 100644 processing/src/main/java/org/apache/druid/frame/channel/ComposingReadableFrameChannel.java create mode 100644 processing/src/main/java/org/apache/druid/frame/channel/ComposingWritableFrameChannel.java create mode 100644 processing/src/main/java/org/apache/druid/frame/processor/ComposingOutputChannelFactory.java diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java index 93d57a815685..956c1b74e31a 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java @@ -637,7 +637,7 @@ private OutputChannelFactory makeStageOutputChannelFactory(final FrameContext fr final File fileChannelDirectory = new File(context.tempDir(), StringUtils.format("output_stage_%06d", stageNumber)); - return new FileOutputChannelFactory(fileChannelDirectory, frameSize); + return new FileOutputChannelFactory(fileChannelDirectory, frameSize, null); } } @@ -662,7 +662,7 @@ private OutputChannelFactory makeSuperSorterIntermediateOutputChannelFactory( } else { final File fileChannelDirectory = new File(tmpDir, StringUtils.format("intermediate_output_stage_%06d", stageNumber)); - return new FileOutputChannelFactory(fileChannelDirectory, frameSize); + return new FileOutputChannelFactory(fileChannelDirectory, frameSize, null); } } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/BroadcastJoinHelperTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/BroadcastJoinHelperTest.java index 4a32144d2745..149576159a2c 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/BroadcastJoinHelperTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/BroadcastJoinHelperTest.java @@ -119,8 +119,8 @@ public void testBuildTableAndInlineData() throws IOException final List channels = new ArrayList<>(); channels.add(new ExplodingReadableFrameChannel()); - channels.add(new ReadableFileFrameChannel(FrameFile.open(testDataFile1))); - channels.add(new ReadableFileFrameChannel(FrameFile.open(testDataFile2))); + channels.add(new ReadableFileFrameChannel(FrameFile.open(testDataFile1, null))); + channels.add(new ReadableFileFrameChannel(FrameFile.open(testDataFile2, null))); final List channelReaders = new ArrayList<>(); channelReaders.add(null); @@ -212,7 +212,7 @@ public void testBuildTableMemoryLimit() throws IOException sideStageChannelNumberMap.put(0, 0); final List channels = new ArrayList<>(); - channels.add(new ReadableFileFrameChannel(FrameFile.open(testDataFile1))); + channels.add(new ReadableFileFrameChannel(FrameFile.open(testDataFile1, null))); final List channelReaders = new ArrayList<>(); channelReaders.add(frameReader1); diff --git a/processing/src/main/java/org/apache/druid/frame/channel/BoundedWritableFrameChannel.java b/processing/src/main/java/org/apache/druid/frame/channel/BoundedWritableFrameChannel.java new file mode 100644 index 000000000000..251d9a6b0a48 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/frame/channel/BoundedWritableFrameChannel.java @@ -0,0 +1,62 @@ +/* + * 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.frame.channel; + +import com.google.common.util.concurrent.ListenableFuture; + +import javax.annotation.Nullable; +import java.io.IOException; + +public class BoundedWritableFrameChannel implements WritableFrameChannel +{ + private final WritableFrameChannel delegate; + private final ByteTracker byteTracker; + + public BoundedWritableFrameChannel(WritableFrameChannel delegate, ByteTracker byteTracker) + { + this.delegate = delegate; + this.byteTracker = byteTracker; + } + + @Override + public void write(FrameWithPartition frameWithPartition) throws IOException + { + byteTracker.reserve(frameWithPartition.frame().numBytes()); + delegate.write(frameWithPartition); + } + + @Override + public void fail(@Nullable Throwable cause) throws IOException + { + delegate.fail(cause); + } + + @Override + public void close() throws IOException + { + delegate.close(); + } + + @Override + public ListenableFuture writabilityFuture() + { + return delegate.writabilityFuture(); + } +} diff --git a/processing/src/main/java/org/apache/druid/frame/channel/ByteTracker.java b/processing/src/main/java/org/apache/druid/frame/channel/ByteTracker.java new file mode 100644 index 000000000000..ee42329b126c --- /dev/null +++ b/processing/src/main/java/org/apache/druid/frame/channel/ByteTracker.java @@ -0,0 +1,55 @@ +/* + * 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.frame.channel; + +import com.google.common.base.Preconditions; +import com.google.errorprone.annotations.concurrent.GuardedBy; +import org.apache.druid.query.ResourceLimitExceededException; + + +public class ByteTracker +{ + @GuardedBy("this") + private final long maxBytes; + + @GuardedBy("this") + private long currentBytes; + + public ByteTracker(long maxBytes) + { + this.maxBytes = maxBytes; + } + + public synchronized void reserve(long byteCount) throws ResourceLimitExceededException + { + Preconditions.checkState(byteCount >= 0, "Can't reserve negative bytes"); + if (currentBytes + byteCount > maxBytes) { + throw new ResourceLimitExceededException(""); + } + currentBytes += byteCount; + } + + public synchronized void release(long byteCount) + { + Preconditions.checkState(byteCount >= 0, "Can't release negative bytes"); + Preconditions.checkState(currentBytes >= byteCount, "Can't release more than used bytes"); + currentBytes -= byteCount; + } +} diff --git a/processing/src/main/java/org/apache/druid/frame/channel/ComposingReadableFrameChannel.java b/processing/src/main/java/org/apache/druid/frame/channel/ComposingReadableFrameChannel.java new file mode 100644 index 000000000000..47b5b4f13a7f --- /dev/null +++ b/processing/src/main/java/org/apache/druid/frame/channel/ComposingReadableFrameChannel.java @@ -0,0 +1,143 @@ +/* + * 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.frame.channel; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import org.apache.druid.frame.Frame; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.function.Supplier; + +public class ComposingReadableFrameChannel implements ReadableFrameChannel +{ + private final List> channels; + private ReadableFrameChannel currentChannel; + private int currentIndex; + + public ComposingReadableFrameChannel( + int partition, + List> channels, + Map> partitionToChannelMap + ) + { + Preconditions.checkNotNull(channels, "channels is null"); + Preconditions.checkNotNull(partitionToChannelMap, "partitionToChannelMap is null"); + if (partitionToChannelMap.get(partition) == null) { + // no writes for the partition, send an empty readable channel + this.channels = ImmutableList.of(() -> ReadableNilFrameChannel.INSTANCE); + } else { + HashSet validChannels = partitionToChannelMap.get(partition); + Preconditions.checkState(validChannels.size() > 0, "No channels found for partition " + partition); + ImmutableList.Builder> validChannelsBuilder = ImmutableList.builder(); + ArrayList sortedChannelIds = new ArrayList<>(validChannels); + Collections.sort(sortedChannelIds); // the data was written from lowest to highest channel + for (Integer channelId : sortedChannelIds) { + validChannelsBuilder.add(channels.get(channelId)); + } + this.channels = validChannelsBuilder.build(); + } + this.currentIndex = 0; + this.currentChannel = null; + } + + @Override + public boolean isFinished() + { + initCurrentChannel(); + if (!currentChannel.isFinished()) { + return false; + } + currentChannel.close(); + currentChannel = null; + if (isLastIndex()) { + return true; + } + ++currentIndex; + return isFinished(); + } + + @Override + public boolean canRead() + { + initCurrentChannel(); + if (currentChannel.canRead()) { + return true; + } + if (currentChannel.isFinished()) { + currentChannel.close(); + currentChannel = null; + if (isLastIndex()) { + return false; + } + ++currentIndex; + return canRead(); + } + return false; + } + + @Override + public Frame read() + { + return currentChannel.read(); + } + + @Override + public ListenableFuture readabilityFuture() + { + initCurrentChannel(); + if (!currentChannel.isFinished()) { + return currentChannel.readabilityFuture(); + } + currentChannel.close(); + currentChannel = null; + if (isLastIndex()) { + return Futures.immediateFuture(true); + } + ++currentIndex; + return readabilityFuture(); + } + + @Override + public void close() + { + if (currentChannel != null) { + currentChannel.close(); + } + } + + private boolean isLastIndex() + { + return currentIndex == channels.size() - 1; + } + + private void initCurrentChannel() + { + if (currentChannel == null) { + currentChannel = channels.get(currentIndex).get(); + } + } +} diff --git a/processing/src/main/java/org/apache/druid/frame/channel/ComposingWritableFrameChannel.java b/processing/src/main/java/org/apache/druid/frame/channel/ComposingWritableFrameChannel.java new file mode 100644 index 000000000000..6abf3ab5dad9 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/frame/channel/ComposingWritableFrameChannel.java @@ -0,0 +1,95 @@ +/* + * 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.frame.channel; + +import com.google.common.base.Preconditions; +import com.google.common.collect.Sets; +import com.google.common.util.concurrent.ListenableFuture; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.query.ResourceLimitExceededException; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.function.Supplier; + +public class ComposingWritableFrameChannel implements WritableFrameChannel +{ + private final List> channels; + private final Map> partitionToChannelMap; + private int currentIndex; + + public ComposingWritableFrameChannel( + List> channels, + Map> partitionToChannelMap + ) + { + this.channels = Preconditions.checkNotNull(channels, "channels is null"); + this.partitionToChannelMap = + Preconditions.checkNotNull(partitionToChannelMap, "partitionToChannelMap is null"); + this.currentIndex = 0; + } + + @Override + public void write(FrameWithPartition frameWithPartition) throws IOException + { + if (currentIndex >= channels.size()) { + throw new ISE("No more channels available to write. Total available channels : " + channels.size()); + } + + try { + channels.get(currentIndex).get().write(frameWithPartition); + partitionToChannelMap.computeIfAbsent(frameWithPartition.partition(), k -> Sets.newHashSetWithExpectedSize(1)) + .add(currentIndex); + } + catch (ResourceLimitExceededException rlee) { + channels.get(currentIndex).get().close(); + currentIndex++; + if (currentIndex >= channels.size()) { + throw rlee; + } + write(frameWithPartition); + } + } + + @Override + public void fail(@Nullable Throwable cause) throws IOException + { + for (Supplier channel : channels) { + channel.get().fail(cause); + } + } + + @Override + public void close() throws IOException + { + if (currentIndex < channels.size()) { + channels.get(currentIndex).get().close(); + } + } + + @Override + public ListenableFuture writabilityFuture() + { + return channels.get(currentIndex).get().writabilityFuture(); + } +} diff --git a/processing/src/main/java/org/apache/druid/frame/file/FrameFile.java b/processing/src/main/java/org/apache/druid/frame/file/FrameFile.java index 1f7c83c1c6f3..da79d35acaa8 100644 --- a/processing/src/main/java/org/apache/druid/frame/file/FrameFile.java +++ b/processing/src/main/java/org/apache/druid/frame/file/FrameFile.java @@ -21,6 +21,7 @@ import org.apache.datasketches.memory.Memory; import org.apache.druid.frame.Frame; +import org.apache.druid.frame.channel.ByteTracker; import org.apache.druid.java.util.common.FileUtils; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.IOE; @@ -139,13 +140,17 @@ private FrameFile( /** * Open a frame file with certain optional flags. - * - * @param file Æ’rame file + * @param file Æ’rame file + * @param byteTracker * @param flags optional flags */ - public static FrameFile open(final File file, final Flag... flags) throws IOException + public static FrameFile open( + final File file, + @Nullable final ByteTracker byteTracker, + final Flag... flags + ) throws IOException { - return open(file, Integer.MAX_VALUE, flags); + return open(file, Integer.MAX_VALUE, byteTracker, flags); } /** @@ -153,12 +158,17 @@ public static FrameFile open(final File file, final Flag... flags) throws IOExce * * Package-private because this method is intended for use in tests. In production, {@code maxMmapSize} is * set to {@link Integer#MAX_VALUE}. - * - * @param file Æ’rame file + * @param file Æ’rame file * @param maxMmapSize largest buffer to mmap at once + * @param byteTracker * @param flags optional flags */ - static FrameFile open(final File file, final int maxMmapSize, final Flag... flags) throws IOException + static FrameFile open( + final File file, + final int maxMmapSize, + @Nullable final ByteTracker byteTracker, + final Flag... flags + ) throws IOException { final EnumSet flagSet = flags.length == 0 ? EnumSet.noneOf(Flag.class) : EnumSet.copyOf(Arrays.asList(flags)); @@ -235,6 +245,9 @@ static FrameFile open(final File file, final int maxMmapSize, final Flag... flag if (!file.delete()) { log.warn("Could not delete frame file [%s]", file); } + if (byteTracker != null) { + byteTracker.release(fileLength); + } }); } diff --git a/processing/src/main/java/org/apache/druid/frame/processor/ComposingOutputChannelFactory.java b/processing/src/main/java/org/apache/druid/frame/processor/ComposingOutputChannelFactory.java new file mode 100644 index 000000000000..dc8f3a20f3df --- /dev/null +++ b/processing/src/main/java/org/apache/druid/frame/processor/ComposingOutputChannelFactory.java @@ -0,0 +1,163 @@ +/* + * 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.frame.processor; + +import com.google.common.base.Preconditions; +import com.google.common.base.Suppliers; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Sets; +import org.apache.druid.frame.allocation.ArenaMemoryAllocator; +import org.apache.druid.frame.channel.ComposingReadableFrameChannel; +import org.apache.druid.frame.channel.ComposingWritableFrameChannel; +import org.apache.druid.frame.channel.PartitionedReadableFrameChannel; +import org.apache.druid.frame.channel.ReadableFrameChannel; +import org.apache.druid.frame.channel.WritableFrameChannel; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.function.Supplier; + +public class ComposingOutputChannelFactory implements OutputChannelFactory +{ + private final List channelFactories; + private final int frameSize; + + public ComposingOutputChannelFactory(List channelFactories, int frameSize) + { + this.channelFactories = Preconditions.checkNotNull(channelFactories, "channelFactories is null"); + this.frameSize = frameSize; + } + + @Override + public OutputChannel openChannel(int partitionNumber) throws IOException + { + ImmutableList.Builder> writableFrameChannelSuppliersBuilder = ImmutableList.builder(); + ImmutableList.Builder> readableFrameChannelSuppliersBuilder = ImmutableList.builder(); + for (OutputChannelFactory channelFactory : channelFactories) { + // open channel lazily + Supplier channel = + Suppliers.memoize(() -> { + try { + return channelFactory.openChannel(partitionNumber); + } + catch (IOException e) { + throw new UncheckedIOException(e); + } + })::get; + writableFrameChannelSuppliersBuilder.add(() -> channel.get().getWritableChannel()); + readableFrameChannelSuppliersBuilder.add(() -> channel.get().getReadableChannelSupplier().get()); + } + Map> partitionToChannelMap = new HashMap<>(); + ComposingWritableFrameChannel writableFrameChannel = new ComposingWritableFrameChannel( + writableFrameChannelSuppliersBuilder.build(), + partitionToChannelMap + ); + Supplier readableFrameChannelSupplier = Suppliers.memoize( + () -> new ComposingReadableFrameChannel( + partitionNumber, + readableFrameChannelSuppliersBuilder.build(), + partitionToChannelMap + ) + )::get; + return OutputChannel.pair( + writableFrameChannel, + ArenaMemoryAllocator.createOnHeap(frameSize), + readableFrameChannelSupplier, + partitionNumber + ); + } + + @Override + public PartitionedOutputChannel openPartitionedChannel(String name, boolean deleteAfterRead) + { + ImmutableList.Builder> writableFrameChannelsBuilder = ImmutableList.builder(); + ImmutableList.Builder> readableFrameChannelSuppliersBuilder = + ImmutableList.builder(); + for (OutputChannelFactory channelFactory : channelFactories) { + Supplier channel = + Suppliers.memoize(() -> { + try { + return channelFactory.openPartitionedChannel(name, deleteAfterRead); + } + catch (IOException e) { + throw new UncheckedIOException(e); + } + })::get; + writableFrameChannelsBuilder.add(() -> channel.get().getWritableChannel()); + readableFrameChannelSuppliersBuilder.add(() -> channel.get().getReadableChannelSupplier().get()); + } + Map> partitionToChannelMap = new HashMap<>(); + ComposingWritableFrameChannel writableFrameChannel = new ComposingWritableFrameChannel( + writableFrameChannelsBuilder.build(), + partitionToChannelMap + ); + + List> readableFrameChannelSuppliers = + readableFrameChannelSuppliersBuilder.build(); + PartitionedReadableFrameChannel partitionedReadableFrameChannel = new PartitionedReadableFrameChannel() + { + private final Set openedChannels = Sets.newHashSetWithExpectedSize(1); + + @Override + public ReadableFrameChannel getReadableFrameChannel(int partitionNumber) + { + ImmutableList.Builder> suppliers = ImmutableList.builder(); + for (int i = 0; i < readableFrameChannelSuppliers.size(); i++) { + int finalI = i; + suppliers.add( + Suppliers.memoize( + () -> { + openedChannels.add(finalI); + return readableFrameChannelSuppliers.get(finalI).get().getReadableFrameChannel(partitionNumber); + } + )::get + ); + } + + return new ComposingReadableFrameChannel(partitionNumber, suppliers.build(), partitionToChannelMap); + } + + @Override + public void close() throws IOException + { + for (Integer channelId : openedChannels) { + readableFrameChannelSuppliers.get(channelId).get().close(); + } + } + }; + + return PartitionedOutputChannel.pair( + writableFrameChannel, + ArenaMemoryAllocator.createOnHeap(frameSize), + () -> partitionedReadableFrameChannel + ); + } + + @Override + public OutputChannel openNilChannel(int partitionNumber) + { + return OutputChannel.nil(partitionNumber); + } +} diff --git a/processing/src/main/java/org/apache/druid/frame/processor/FileOutputChannelFactory.java b/processing/src/main/java/org/apache/druid/frame/processor/FileOutputChannelFactory.java index 34e01f63d884..8a7514c0bb7c 100644 --- a/processing/src/main/java/org/apache/druid/frame/processor/FileOutputChannelFactory.java +++ b/processing/src/main/java/org/apache/druid/frame/processor/FileOutputChannelFactory.java @@ -22,9 +22,12 @@ import com.google.common.base.Suppliers; import org.apache.druid.frame.Frame; import org.apache.druid.frame.allocation.ArenaMemoryAllocator; +import org.apache.druid.frame.channel.BoundedWritableFrameChannel; +import org.apache.druid.frame.channel.ByteTracker; import org.apache.druid.frame.channel.PartitionedReadableFrameChannel; import org.apache.druid.frame.channel.ReadableFileFrameChannel; import org.apache.druid.frame.channel.ReadableFrameChannel; +import org.apache.druid.frame.channel.WritableFrameChannel; import org.apache.druid.frame.channel.WritableFrameFileChannel; import org.apache.druid.frame.file.FrameFile; import org.apache.druid.frame.file.FrameFileWriter; @@ -47,11 +50,17 @@ public class FileOutputChannelFactory implements OutputChannelFactory { private final File fileChannelsDirectory; private final int frameSize; + private final ByteTracker byteTracker; - public FileOutputChannelFactory(final File fileChannelsDirectory, final int frameSize) + public FileOutputChannelFactory( + final File fileChannelsDirectory, + final int frameSize, + final ByteTracker byteTracker + ) { this.fileChannelsDirectory = fileChannelsDirectory; this.frameSize = frameSize; + this.byteTracker = byteTracker == null ? new ByteTracker(Long.MAX_VALUE) : byteTracker; } @Override @@ -62,22 +71,25 @@ public OutputChannel openChannel(int partitionNumber) throws IOException final String fileName = StringUtils.format("part_%06d_%s", partitionNumber, UUID.randomUUID().toString()); final File file = new File(fileChannelsDirectory, fileName); - final WritableFrameFileChannel writableChannel = - new WritableFrameFileChannel( - FrameFileWriter.open( - Files.newByteChannel( - file.toPath(), - StandardOpenOption.CREATE_NEW, - StandardOpenOption.WRITE - ), - ByteBuffer.allocate(Frame.compressionBufferSize(frameSize)) - ) + final WritableFrameChannel writableChannel = + new BoundedWritableFrameChannel( + new WritableFrameFileChannel( + FrameFileWriter.open( + Files.newByteChannel( + file.toPath(), + StandardOpenOption.CREATE_NEW, + StandardOpenOption.WRITE + ), + ByteBuffer.allocate(Frame.compressionBufferSize(frameSize)) + ) + ), + byteTracker ); final Supplier readableChannelSupplier = Suppliers.memoize( () -> { try { - final FrameFile frameFile = FrameFile.open(file, FrameFile.Flag.DELETE_ON_CLOSE); + final FrameFile frameFile = FrameFile.open(file, byteTracker, FrameFile.Flag.DELETE_ON_CLOSE); return new ReadableFileFrameChannel(frameFile); } catch (IOException e) { @@ -99,21 +111,25 @@ public PartitionedOutputChannel openPartitionedChannel(String name, boolean dele { FileUtils.mkdirp(fileChannelsDirectory); final File file = new File(fileChannelsDirectory, name); - WritableFrameFileChannel writableFrameFileChannel = new WritableFrameFileChannel( - FrameFileWriter.open( - Files.newByteChannel( - file.toPath(), - StandardOpenOption.CREATE_NEW, - StandardOpenOption.WRITE - ), - ByteBuffer.allocate(Frame.compressionBufferSize(frameSize)) - ) - ); + WritableFrameChannel writableFrameFileChannel = + new BoundedWritableFrameChannel( + new WritableFrameFileChannel( + FrameFileWriter.open( + Files.newByteChannel( + file.toPath(), + StandardOpenOption.CREATE_NEW, + StandardOpenOption.WRITE + ), + ByteBuffer.allocate(Frame.compressionBufferSize(frameSize)) + ) + ), + byteTracker + ); Supplier frameFileSupplier = Suppliers.memoize( () -> { try { - return deleteAfterRead ? FrameFile.open(file, FrameFile.Flag.DELETE_ON_CLOSE) - : FrameFile.open(file); + return deleteAfterRead ? FrameFile.open(file, byteTracker, FrameFile.Flag.DELETE_ON_CLOSE) + : FrameFile.open(file, byteTracker); } catch (IOException e) { throw new UncheckedIOException(e); diff --git a/processing/src/main/java/org/apache/druid/frame/processor/OutputChannel.java b/processing/src/main/java/org/apache/druid/frame/processor/OutputChannel.java index 086dbac8bfa2..779fa19c73a5 100644 --- a/processing/src/main/java/org/apache/druid/frame/processor/OutputChannel.java +++ b/processing/src/main/java/org/apache/druid/frame/processor/OutputChannel.java @@ -132,6 +132,11 @@ public ReadableFrameChannel getReadableChannel() return readableChannelSupplier.get(); } + public Supplier getReadableChannelSupplier() + { + return readableChannelSupplier; + } + public int getPartitionNumber() { return partitionNumber; diff --git a/processing/src/test/java/org/apache/druid/frame/file/FrameFileTest.java b/processing/src/test/java/org/apache/druid/frame/file/FrameFileTest.java index d2168493d00a..79cbe74ba28f 100644 --- a/processing/src/test/java/org/apache/druid/frame/file/FrameFileTest.java +++ b/processing/src/test/java/org/apache/druid/frame/file/FrameFileTest.java @@ -211,7 +211,7 @@ public IntObjectPair apply(final Frame frame) @Test public void test_numFrames() throws IOException { - try (final FrameFile frameFile = FrameFile.open(file, maxMmapSize)) { + try (final FrameFile frameFile = FrameFile.open(file, maxMmapSize, null)) { Assert.assertEquals(computeExpectedNumFrames(), frameFile.numFrames()); } } @@ -219,7 +219,7 @@ public void test_numFrames() throws IOException @Test public void test_numPartitions() throws IOException { - try (final FrameFile frameFile = FrameFile.open(file, maxMmapSize)) { + try (final FrameFile frameFile = FrameFile.open(file, maxMmapSize, null)) { Assert.assertEquals(computeExpectedNumPartitions(), frameFile.numPartitions()); } } @@ -227,7 +227,7 @@ public void test_numPartitions() throws IOException @Test public void test_frame_first() throws IOException { - try (final FrameFile frameFile = FrameFile.open(file, maxMmapSize)) { + try (final FrameFile frameFile = FrameFile.open(file, maxMmapSize, null)) { // Skip test for empty files. Assume.assumeThat(frameFile.numFrames(), Matchers.greaterThan(0)); @@ -239,7 +239,7 @@ public void test_frame_first() throws IOException @Test public void test_frame_last() throws IOException { - try (final FrameFile frameFile = FrameFile.open(file, maxMmapSize)) { + try (final FrameFile frameFile = FrameFile.open(file, maxMmapSize, null)) { // Skip test for empty files. Assume.assumeThat(frameFile.numFrames(), Matchers.greaterThan(0)); @@ -256,7 +256,7 @@ public void test_frame_last() throws IOException @Test public void test_frame_outOfBoundsNegative() throws IOException { - try (final FrameFile frameFile = FrameFile.open(file, maxMmapSize)) { + try (final FrameFile frameFile = FrameFile.open(file, maxMmapSize, null)) { expectedException.expect(IllegalArgumentException.class); expectedException.expectMessage("Frame [-1] out of bounds"); frameFile.frame(-1); @@ -266,7 +266,7 @@ public void test_frame_outOfBoundsNegative() throws IOException @Test public void test_frame_outOfBoundsTooLarge() throws IOException { - try (final FrameFile frameFile = FrameFile.open(file, maxMmapSize)) { + try (final FrameFile frameFile = FrameFile.open(file, maxMmapSize, null)) { expectedException.expect(IllegalArgumentException.class); expectedException.expectMessage(StringUtils.format("Frame [%,d] out of bounds", frameFile.numFrames())); frameFile.frame(frameFile.numFrames()); @@ -278,7 +278,7 @@ public void test_frame_readAllDataViaStorageAdapter() throws IOException { final FrameReader frameReader = FrameReader.create(adapter.getRowSignature()); - try (final FrameFile frameFile = FrameFile.open(file, maxMmapSize)) { + try (final FrameFile frameFile = FrameFile.open(file, maxMmapSize, null)) { final Sequence> frameFileRows = Sequences.concat( () -> IntStream.range(0, frameFile.numFrames()) .mapToObj(frameFile::frame) @@ -295,7 +295,7 @@ public void test_frame_readAllDataViaStorageAdapter() throws IOException @Test public void test_getPartitionStartFrame() throws IOException { - try (final FrameFile frameFile = FrameFile.open(file, maxMmapSize)) { + try (final FrameFile frameFile = FrameFile.open(file, maxMmapSize, null)) { if (partitioned) { for (int partitionNum = 0; partitionNum < frameFile.numPartitions(); partitionNum++) { Assert.assertEquals( @@ -320,7 +320,7 @@ public void test_getPartitionStartFrame() throws IOException @Test public void test_file() throws IOException { - try (final FrameFile frameFile = FrameFile.open(file, maxMmapSize)) { + try (final FrameFile frameFile = FrameFile.open(file, maxMmapSize, null)) { Assert.assertEquals(file, frameFile.file()); } } @@ -328,17 +328,17 @@ public void test_file() throws IOException @Test public void test_open_withDeleteOnClose() throws IOException { - FrameFile.open(file, maxMmapSize).close(); + FrameFile.open(file, maxMmapSize, null).close(); Assert.assertTrue(file.exists()); - FrameFile.open(file, FrameFile.Flag.DELETE_ON_CLOSE).close(); + FrameFile.open(file, null, FrameFile.Flag.DELETE_ON_CLOSE).close(); Assert.assertFalse(file.exists()); } @Test public void test_newReference() throws IOException { - final FrameFile frameFile1 = FrameFile.open(file, FrameFile.Flag.DELETE_ON_CLOSE); + final FrameFile frameFile1 = FrameFile.open(file, null, FrameFile.Flag.DELETE_ON_CLOSE); final FrameFile frameFile2 = frameFile1.newReference(); final FrameFile frameFile3 = frameFile2.newReference(); diff --git a/processing/src/test/java/org/apache/druid/frame/file/FrameFileWriterTest.java b/processing/src/test/java/org/apache/druid/frame/file/FrameFileWriterTest.java index 5b5687a60ce3..dbc8e484de18 100644 --- a/processing/src/test/java/org/apache/druid/frame/file/FrameFileWriterTest.java +++ b/processing/src/test/java/org/apache/druid/frame/file/FrameFileWriterTest.java @@ -70,7 +70,7 @@ public void test_abort_afterAllFrames() throws IOException fileWriter.abort(); - final IllegalStateException e = Assert.assertThrows(IllegalStateException.class, () -> FrameFile.open(file)); + final IllegalStateException e = Assert.assertThrows(IllegalStateException.class, () -> FrameFile.open(file, null)); MatcherAssert.assertThat( e, diff --git a/processing/src/test/java/org/apache/druid/frame/processor/FrameProcessorExecutorTest.java b/processing/src/test/java/org/apache/druid/frame/processor/FrameProcessorExecutorTest.java index 349fdb2efe23..81afa21dcbdf 100644 --- a/processing/src/test/java/org/apache/druid/frame/processor/FrameProcessorExecutorTest.java +++ b/processing/src/test/java/org/apache/druid/frame/processor/FrameProcessorExecutorTest.java @@ -150,7 +150,7 @@ public void test_runFully() throws Exception Assert.assertEquals( adapter.getNumRows() * 2, FrameTestUtil.readRowsFromFrameChannel( - new ReadableFileFrameChannel(FrameFile.open(outFile)), + new ReadableFileFrameChannel(FrameFile.open(outFile, null)), FrameReader.create(adapter.getRowSignature()) ).toList().size() ); @@ -482,7 +482,7 @@ public void accept(Frame frame) private static ReadableFrameChannel openFileChannel(final File file) { try { - return new ReadableFileFrameChannel(FrameFile.open(file)); + return new ReadableFileFrameChannel(FrameFile.open(file, null)); } catch (IOException e) { throw new RuntimeException(e); diff --git a/processing/src/test/java/org/apache/druid/frame/processor/SuperSorterTest.java b/processing/src/test/java/org/apache/druid/frame/processor/SuperSorterTest.java index ff8e65be29f2..ac8db8bdef37 100644 --- a/processing/src/test/java/org/apache/druid/frame/processor/SuperSorterTest.java +++ b/processing/src/test/java/org/apache/druid/frame/processor/SuperSorterTest.java @@ -27,6 +27,7 @@ import org.apache.druid.frame.FrameType; import org.apache.druid.frame.allocation.ArenaMemoryAllocator; import org.apache.druid.frame.channel.BlockingQueueFrameChannel; +import org.apache.druid.frame.channel.ByteTracker; import org.apache.druid.frame.channel.ReadableFileFrameChannel; import org.apache.druid.frame.channel.ReadableFrameChannel; import org.apache.druid.frame.channel.WritableFrameChannel; @@ -137,7 +138,7 @@ private void testSingleEmptyInputChannel(boolean isDurableStorage) throws Except ClusterBy.none(), outputPartitionsFuture, exec, - new FileOutputChannelFactory(tempFolder, FRAME_SIZE), + new FileOutputChannelFactory(tempFolder, FRAME_SIZE, null), isDurableStorage ? new DurableStorageOutputChannelFactory( "0", 0, @@ -146,7 +147,7 @@ private void testSingleEmptyInputChannel(boolean isDurableStorage) throws Except FRAME_SIZE, new LocalFileStorageConnector(tempFolder), tempFolder - ) : new FileOutputChannelFactory(tempFolder, FRAME_SIZE), + ) : new FileOutputChannelFactory(tempFolder, FRAME_SIZE, null), 2, 2, -1, @@ -181,7 +182,7 @@ public static class ParameterizedCasesTest extends InitializedNullHandlingTest private final int maxActiveProcessors; private final int maxChannelsPerProcessor; private final int numThreads; - private final boolean isDurableStorage; + private final boolean isComposedStorage; private StorageAdapter adapter; private RowSignature signature; @@ -196,7 +197,7 @@ public ParameterizedCasesTest( int maxActiveProcessors, int maxChannelsPerProcessor, int numThreads, - boolean isDurableStorage + boolean isComposedStorage ) { this.maxRowsPerFrame = maxRowsPerFrame; @@ -205,7 +206,7 @@ public ParameterizedCasesTest( this.maxActiveProcessors = maxActiveProcessors; this.maxChannelsPerProcessor = maxChannelsPerProcessor; this.numThreads = numThreads; - this.isDurableStorage = isDurableStorage; + this.isComposedStorage = isComposedStorage; } @Parameterized.Parameters( @@ -215,7 +216,7 @@ public ParameterizedCasesTest( + "maxActiveProcessors = {3}, " + "maxChannelsPerProcessor = {4}, " + "numThreads = {5}, " - + "isDurableStorage = {6}" + + "isComposedStorage = {6}" ) public static Iterable constructorFeeder() { @@ -227,7 +228,7 @@ public static Iterable constructorFeeder() for (int maxActiveProcessors : new int[]{1, 2, 4}) { for (int maxChannelsPerProcessor : new int[]{2, 3, 8}) { for (int numThreads : new int[]{1, 3}) { - for (boolean isDurableStorage : new boolean[]{true, false}) { + for (boolean isComposedStorage : new boolean[]{true, false}) { if (maxActiveProcessors >= maxChannelsPerProcessor) { constructors.add( new Object[]{ @@ -237,7 +238,7 @@ public static Iterable constructorFeeder() maxActiveProcessors, maxChannelsPerProcessor, numThreads, - isDurableStorage + isComposedStorage } ); } @@ -302,15 +303,21 @@ private OutputChannels verifySuperSorter( ) throws Exception { final File tempFolder = temporaryFolder.newFolder(); - final OutputChannelFactory outputChannelFactory = isDurableStorage ? new DurableStorageOutputChannelFactory( - "0", - 0, - 0, - "0", - maxBytesPerFrame, - new LocalFileStorageConnector(tempFolder), - tempFolder - ) : new FileOutputChannelFactory(tempFolder, maxBytesPerFrame); + final OutputChannelFactory outputChannelFactory = isComposedStorage ? new ComposingOutputChannelFactory( + ImmutableList.of( + new FileOutputChannelFactory(tempFolder, maxBytesPerFrame, new ByteTracker(maxBytesPerFrame)), + new DurableStorageOutputChannelFactory( + "0", + 0, + 0, + "0", + maxBytesPerFrame, + new LocalFileStorageConnector(tempFolder), + tempFolder + ) + ), + maxBytesPerFrame + ) : new FileOutputChannelFactory(tempFolder, maxBytesPerFrame, null); final RowKeyReader keyReader = clusterBy.keyReader(signature); final Comparator keyComparator = clusterBy.keyComparator(); final SettableFuture clusterByPartitionsFuture = SettableFuture.create(); @@ -322,7 +329,7 @@ private OutputChannels verifySuperSorter( clusterBy, clusterByPartitionsFuture, exec, - new FileOutputChannelFactory(tempFolder, maxBytesPerFrame), + new FileOutputChannelFactory(tempFolder, maxBytesPerFrame, null), outputChannelFactory, maxActiveProcessors, maxChannelsPerProcessor, @@ -718,7 +725,7 @@ public void accept(final Frame frame) for (int i = 0; i < writableChannels.size(); i++) { WritableFrameChannel writableChannel = writableChannels.get(i); writableChannel.close(); - retVal.add(new ReadableFileFrameChannel(FrameFile.open(files.get(i)))); + retVal.add(new ReadableFileFrameChannel(FrameFile.open(files.get(i), null))); } return retVal; From 5ea228d72977fd4339fc565dba95fd5d784a83d4 Mon Sep 17 00:00:00 2001 From: Rohan Garg Date: Mon, 5 Dec 2022 18:27:41 +0530 Subject: [PATCH 10/15] Test output channel factories (cherry picked from commit 17fc776fdbaa2be580211aff04dd41577f1ddcb8) --- .../ComposingOutputChannelFactory.java | 14 ++ ...BlockingQueueOutputChannelFactoryTest.java | 24 +-- .../ComposingOutputChannelFactoryTest.java | 127 +++++++++++++ ...urableStorageOutputChannelFactoryTest.java | 49 +++++ .../FileOutputChannelFactoryTest.java | 36 ++++ .../processor/OutputChannelFactoryTest.java | 168 ++++++++++++++++++ 6 files changed, 400 insertions(+), 18 deletions(-) create mode 100644 processing/src/test/java/org/apache/druid/frame/processor/ComposingOutputChannelFactoryTest.java create mode 100644 processing/src/test/java/org/apache/druid/frame/processor/DurableStorageOutputChannelFactoryTest.java create mode 100644 processing/src/test/java/org/apache/druid/frame/processor/FileOutputChannelFactoryTest.java create mode 100644 processing/src/test/java/org/apache/druid/frame/processor/OutputChannelFactoryTest.java diff --git a/processing/src/main/java/org/apache/druid/frame/processor/ComposingOutputChannelFactory.java b/processing/src/main/java/org/apache/druid/frame/processor/ComposingOutputChannelFactory.java index dc8f3a20f3df..8d4d8b6c20ab 100644 --- a/processing/src/main/java/org/apache/druid/frame/processor/ComposingOutputChannelFactory.java +++ b/processing/src/main/java/org/apache/druid/frame/processor/ComposingOutputChannelFactory.java @@ -39,6 +39,9 @@ import java.util.Set; import java.util.function.Supplier; +/** + * + */ public class ComposingOutputChannelFactory implements OutputChannelFactory { private final List channelFactories; @@ -69,6 +72,9 @@ public OutputChannel openChannel(int partitionNumber) throws IOException writableFrameChannelSuppliersBuilder.add(() -> channel.get().getWritableChannel()); readableFrameChannelSuppliersBuilder.add(() -> channel.get().getReadableChannelSupplier().get()); } + + // the map maintains a mapping of channels which have the data for a given partition. + // it is useful to identify the readable channels to open in the composition while reading the partition data. Map> partitionToChannelMap = new HashMap<>(); ComposingWritableFrameChannel writableFrameChannel = new ComposingWritableFrameChannel( writableFrameChannelSuppliersBuilder.build(), @@ -108,6 +114,9 @@ public PartitionedOutputChannel openPartitionedChannel(String name, boolean dele writableFrameChannelsBuilder.add(() -> channel.get().getWritableChannel()); readableFrameChannelSuppliersBuilder.add(() -> channel.get().getReadableChannelSupplier().get()); } + // the map maintains a mapping of channels which have the data for a given partition. + // it is useful to identify the readable channels to open in the composition while reading the partition data. + Map> partitionToChannelMap = new HashMap<>(); ComposingWritableFrameChannel writableFrameChannel = new ComposingWritableFrameChannel( writableFrameChannelsBuilder.build(), @@ -118,6 +127,7 @@ public PartitionedOutputChannel openPartitionedChannel(String name, boolean dele readableFrameChannelSuppliersBuilder.build(); PartitionedReadableFrameChannel partitionedReadableFrameChannel = new PartitionedReadableFrameChannel() { + // maintained so that we only close channels which were opened for reading private final Set openedChannels = Sets.newHashSetWithExpectedSize(1); @Override @@ -158,6 +168,10 @@ public void close() throws IOException @Override public OutputChannel openNilChannel(int partitionNumber) { + // Doing this since some output channel factories create marker objects for nil channels + for (OutputChannelFactory outputChannelFactory : channelFactories) { + outputChannelFactory.openNilChannel(partitionNumber); + } return OutputChannel.nil(partitionNumber); } } diff --git a/processing/src/test/java/org/apache/druid/frame/processor/BlockingQueueOutputChannelFactoryTest.java b/processing/src/test/java/org/apache/druid/frame/processor/BlockingQueueOutputChannelFactoryTest.java index 4911591580a6..421ac709e9a9 100644 --- a/processing/src/test/java/org/apache/druid/frame/processor/BlockingQueueOutputChannelFactoryTest.java +++ b/processing/src/test/java/org/apache/druid/frame/processor/BlockingQueueOutputChannelFactoryTest.java @@ -19,31 +19,19 @@ package org.apache.druid.frame.processor; -import org.junit.Assert; import org.junit.Test; -public class BlockingQueueOutputChannelFactoryTest +public class BlockingQueueOutputChannelFactoryTest extends OutputChannelFactoryTest { - @Test - public void test_openChannel() + public BlockingQueueOutputChannelFactoryTest() { - final int frameSize = 100; - final BlockingQueueOutputChannelFactory factory = new BlockingQueueOutputChannelFactory(frameSize); - final OutputChannel channel = factory.openChannel(1); - - Assert.assertEquals(1, channel.getPartitionNumber()); - Assert.assertEquals(frameSize, channel.getFrameMemoryAllocator().capacity()); + super(new BlockingQueueOutputChannelFactory(100), 100); } + @Override @Test - public void test_openNilChannel() + public void test_openPartitionedChannel() { - final int frameSize = 100; - final BlockingQueueOutputChannelFactory factory = new BlockingQueueOutputChannelFactory(frameSize); - final OutputChannel channel = factory.openNilChannel(1); - - Assert.assertEquals(1, channel.getPartitionNumber()); - Assert.assertTrue(channel.getReadableChannel().isFinished()); - Assert.assertThrows(IllegalStateException.class, channel::getWritableChannel); + // Do nothing since partitioned channels aren't implemented for BlockingQueueOutputChannelFactory yet. } } diff --git a/processing/src/test/java/org/apache/druid/frame/processor/ComposingOutputChannelFactoryTest.java b/processing/src/test/java/org/apache/druid/frame/processor/ComposingOutputChannelFactoryTest.java new file mode 100644 index 000000000000..ebe5d4f7537a --- /dev/null +++ b/processing/src/test/java/org/apache/druid/frame/processor/ComposingOutputChannelFactoryTest.java @@ -0,0 +1,127 @@ +/* + * 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.frame.processor; + +import com.google.common.collect.ImmutableList; +import org.apache.druid.frame.channel.ByteTracker; +import org.apache.druid.frame.channel.FrameWithPartition; +import org.apache.druid.frame.channel.WritableFrameChannel; +import org.apache.druid.storage.local.LocalFileStorageConnector; +import org.junit.Assert; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.io.IOException; +import java.util.concurrent.ExecutionException; + +public class ComposingOutputChannelFactoryTest extends OutputChannelFactoryTest +{ + @ClassRule + public static TemporaryFolder folder = new TemporaryFolder(); + + public ComposingOutputChannelFactoryTest() throws IOException + { + super(new ComposingOutputChannelFactory( + ImmutableList.of( + // TODO : currently hardcoded 256k since it allows one frame to be written to each factory + // nicer to do that automatically + new FileOutputChannelFactory(folder.newFolder(), 100, new ByteTracker(256_000)), + new DurableStorageOutputChannelFactory( + "0", + 0, + 0, + "0", + 100, + new LocalFileStorageConnector(folder.newFolder()), + folder.newFolder() + ) + ), + 100 + ), 100); + } + + @Test + public void test_openChannel2() throws IOException, ExecutionException, InterruptedException + { + ComposingOutputChannelFactory outputChannelFactory = new ComposingOutputChannelFactory( + ImmutableList.of( + new FileOutputChannelFactory(folder.newFolder(), 100, new ByteTracker(1)), + new ThrowingOutputChannelFactory() // adding this to check if it gets called + ), + 100 + ); + OutputChannel channel = outputChannelFactory.openChannel(1); + + Assert.assertEquals(1, channel.getPartitionNumber()); + WritableFrameChannel writableFrameChannel = channel.getWritableChannel(); + writableFrameChannel.writabilityFuture().get(); + Assert.assertThrows( + UnsupportedOperationException.class, + () -> writableFrameChannel.write(new FrameWithPartition(frame, 1)) + ); + } + + @Test + public void test_openChannel3() throws IOException, ExecutionException, InterruptedException + { + // this test checks that the throwing output channels are never opened since 1MB limit on file output channel + // can handle the test data frames + ComposingOutputChannelFactory outputChannelFactory = new ComposingOutputChannelFactory( + ImmutableList.of( + new FileOutputChannelFactory(folder.newFolder(), 100, new ByteTracker(1_000_000)), + new ThrowingOutputChannelFactory() + ), + 100 + ); + OutputChannel channel = outputChannelFactory.openChannel(1); + + Assert.assertEquals(1, channel.getPartitionNumber()); + WritableFrameChannel writableFrameChannel = channel.getWritableChannel(); + writableFrameChannel.writabilityFuture().get(); + writableFrameChannel.write(new FrameWithPartition(frame, 1)); + writableFrameChannel.close(); + + verifySingleFrameReadableChannel(channel.getReadableChannel(), sourceAdapter); + } + + + private static class ThrowingOutputChannelFactory implements OutputChannelFactory + { + + @Override + public OutputChannel openChannel(int partitionNumber) throws IOException + { + throw new UnsupportedOperationException(); + } + + @Override + public PartitionedOutputChannel openPartitionedChannel(String name, boolean deleteAfterRead) + { + throw new UnsupportedOperationException(); + } + + @Override + public OutputChannel openNilChannel(int partitionNumber) + { + throw new UnsupportedOperationException(); + } + } +} diff --git a/processing/src/test/java/org/apache/druid/frame/processor/DurableStorageOutputChannelFactoryTest.java b/processing/src/test/java/org/apache/druid/frame/processor/DurableStorageOutputChannelFactoryTest.java new file mode 100644 index 000000000000..e5cb1d2224cc --- /dev/null +++ b/processing/src/test/java/org/apache/druid/frame/processor/DurableStorageOutputChannelFactoryTest.java @@ -0,0 +1,49 @@ +/* + * 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.frame.processor; + +import org.apache.druid.storage.local.LocalFileStorageConnector; +import org.junit.ClassRule; +import org.junit.rules.TemporaryFolder; + +import java.io.IOException; + +public class DurableStorageOutputChannelFactoryTest extends OutputChannelFactoryTest +{ + @ClassRule + public static TemporaryFolder folder = new TemporaryFolder(); + + public DurableStorageOutputChannelFactoryTest() + throws IOException + { + super( + new DurableStorageOutputChannelFactory( + "0", + 0, + 0, + "0", + 100, + new LocalFileStorageConnector(folder.newFolder()), + folder.newFolder() + ), + 100 + ); + } +} diff --git a/processing/src/test/java/org/apache/druid/frame/processor/FileOutputChannelFactoryTest.java b/processing/src/test/java/org/apache/druid/frame/processor/FileOutputChannelFactoryTest.java new file mode 100644 index 000000000000..1a0038479d38 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/frame/processor/FileOutputChannelFactoryTest.java @@ -0,0 +1,36 @@ +/* + * 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.frame.processor; + +import org.junit.ClassRule; +import org.junit.rules.TemporaryFolder; + +import java.io.IOException; + +public class FileOutputChannelFactoryTest extends OutputChannelFactoryTest +{ + @ClassRule + public static TemporaryFolder folder = new TemporaryFolder(); + + public FileOutputChannelFactoryTest() throws IOException + { + super(new FileOutputChannelFactory(folder.newFolder(), 100, null), 100); + } +} diff --git a/processing/src/test/java/org/apache/druid/frame/processor/OutputChannelFactoryTest.java b/processing/src/test/java/org/apache/druid/frame/processor/OutputChannelFactoryTest.java new file mode 100644 index 000000000000..16d155f3df36 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/frame/processor/OutputChannelFactoryTest.java @@ -0,0 +1,168 @@ +/* + * 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.frame.processor; + +import com.google.common.collect.Iterables; +import org.apache.druid.frame.Frame; +import org.apache.druid.frame.FrameType; +import org.apache.druid.frame.channel.FrameWithPartition; +import org.apache.druid.frame.channel.PartitionedReadableFrameChannel; +import org.apache.druid.frame.channel.ReadableFrameChannel; +import org.apache.druid.frame.channel.WritableFrameChannel; +import org.apache.druid.frame.read.FrameReader; +import org.apache.druid.frame.segment.FrameStorageAdapter; +import org.apache.druid.frame.testutil.FrameSequenceBuilder; +import org.apache.druid.frame.testutil.FrameTestUtil; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.TestIndex; +import org.apache.druid.segment.VirtualColumns; +import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter; +import org.apache.druid.testing.InitializedNullHandlingTest; +import org.junit.Assert; +import org.junit.Test; + +import java.io.IOException; +import java.util.List; +import java.util.concurrent.ExecutionException; +import java.util.function.Supplier; + +public abstract class OutputChannelFactoryTest extends InitializedNullHandlingTest +{ + private final OutputChannelFactory outputChannelFactory; + private final long frameSize; + protected final StorageAdapter sourceAdapter; + protected final Frame frame; + + public OutputChannelFactoryTest(OutputChannelFactory outputChannelFactory, long frameSize) + { + this.outputChannelFactory = outputChannelFactory; + this.frameSize = frameSize; + this.sourceAdapter = new IncrementalIndexStorageAdapter(TestIndex.getIncrementalTestIndex()); + this.frame = Iterables.getOnlyElement(FrameSequenceBuilder.fromAdapter(sourceAdapter) + .frameType(FrameType.COLUMNAR) + .frames() + .toList()); + } + + @Test + public void test_openChannel() throws IOException, ExecutionException, InterruptedException + { + OutputChannel channel = outputChannelFactory.openChannel(1); + + Assert.assertEquals(1, channel.getPartitionNumber()); + + // write data to the channel + WritableFrameChannel writableFrameChannel = channel.getWritableChannel(); + writableFrameChannel.writabilityFuture().get(); + writableFrameChannel.write(new FrameWithPartition(frame, 1)); + writableFrameChannel.close(); + + // read back data from the channel + verifySingleFrameReadableChannel( + channel.getReadableChannel(), + sourceAdapter + ); + Assert.assertEquals(frameSize, channel.getFrameMemoryAllocator().capacity()); + } + + @Test + public void test_openPartitionedChannel() throws IOException, ExecutionException, InterruptedException + { + PartitionedOutputChannel channel = outputChannelFactory.openPartitionedChannel("test", true); + int[] partitions = new int[]{1, 2}; + + // write data to the channel + WritableFrameChannel writableFrameChannel = channel.getWritableChannel(); + writableFrameChannel.writabilityFuture().get(); + for (int partition : partitions) { + writableFrameChannel.write(new FrameWithPartition(frame, partition)); + } + writableFrameChannel.close(); + + // read back data from the channel + Supplier partitionedReadableFrameChannelSupplier = channel.getReadableChannelSupplier(); + for (int partition : partitions) { + verifySingleFrameReadableChannel( + partitionedReadableFrameChannelSupplier.get().getReadableFrameChannel(partition), + sourceAdapter + ); + Assert.assertEquals(frameSize, channel.getFrameMemoryAllocator().capacity()); + } + } + + protected void verifySingleFrameReadableChannel( + ReadableFrameChannel readableFrameChannel, + StorageAdapter adapter + ) throws ExecutionException, InterruptedException + { + readableFrameChannel.readabilityFuture().get(); + // TODO : this is bad. but it is because the input stream channel doesn't honor the contract of readibility future + // either add timeout to the loop or fix input stream channel + while (true) { + if (readableFrameChannel.canRead()) { + break; + } + } + Frame readbackFrame = readableFrameChannel.read(); + readableFrameChannel.readabilityFuture().get(); + Assert.assertFalse(readableFrameChannel.canRead()); + Assert.assertTrue(readableFrameChannel.isFinished()); + readableFrameChannel.close(); + + // build list of rows from written and read data to verify + List> writtenData = adapter.makeCursors( + null, + Intervals.ETERNITY, + VirtualColumns.EMPTY, + Granularities.ALL, + false, + null + ).flatMap(cursor -> FrameTestUtil.readRowsFromCursor(cursor, adapter.getRowSignature())).toList(); + + FrameStorageAdapter frameStorageAdapter = new FrameStorageAdapter( + readbackFrame, + FrameReader.create(adapter.getRowSignature()), + Intervals.ETERNITY + ); + List> readData = frameStorageAdapter.makeCursors( + null, + Intervals.ETERNITY, + VirtualColumns.EMPTY, + Granularities.ALL, + false, + null + ).flatMap(cursor -> FrameTestUtil.readRowsFromCursor(cursor, adapter.getRowSignature())).toList(); + + Assert.assertEquals("Read rows count is different from written rows count", writtenData.size(), readData.size()); + Assert.assertEquals("Read data is different from written data", writtenData, readData); + } + + @Test + public void test_openNilChannel() + { + final OutputChannel channel = outputChannelFactory.openNilChannel(1); + + Assert.assertEquals(1, channel.getPartitionNumber()); + Assert.assertTrue(channel.getReadableChannel().isFinished()); + Assert.assertThrows(IllegalStateException.class, channel::getWritableChannel); + } +} From 8ac983ad7ffa6ec56a4a9da5fee024e9f5d5d8e6 Mon Sep 17 00:00:00 2001 From: Rohan Garg Date: Mon, 5 Dec 2022 21:24:05 +0530 Subject: [PATCH 11/15] Add flags for composition and local storage limit (cherry picked from commit e0809dbc4252f61cf798ff85683b540bf29fb05e) --- .../apache/druid/msq/exec/ControllerImpl.java | 23 ++++++++++-- .../org/apache/druid/msq/exec/WorkerImpl.java | 37 +++++++++++++++++-- .../msq/indexing/MSQWorkerTaskLauncher.java | 26 +++++-------- .../msq/util/MultiStageQueryContext.java | 22 +++++++++++ .../apache/druid/msq/exec/MSQTasksTest.java | 4 +- .../apache/druid/msq/test/MSQTestBase.java | 2 + 6 files changed, 88 insertions(+), 26 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java index 6856c17419b1..dada29baaff0 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java @@ -561,13 +561,30 @@ private QueryDefinition initializeQueryDefAndState(final Closer closer) .orElse(MSQWarnings.DEFAULT_MAX_PARSE_EXCEPTIONS_ALLOWED); } - + ImmutableMap.Builder taskContextOverridesBuilder = ImmutableMap.builder(); + taskContextOverridesBuilder + .put( + MultiStageQueryContext.CTX_ENABLE_DURABLE_SHUFFLE_STORAGE, + isDurableStageStorageEnabled + ).put( + MultiStageQueryContext.CTX_COMPOSED_INTERMEDIATE_SUPER_SORTER_STORAGE, + MultiStageQueryContext.isComposedIntermediateSuperSorterStorageEnabled( + task.getQuerySpec().getQuery().context() + ) + ).put( + MultiStageQueryContext.CTX_INTERMEDIATE_SUPER_SORTER_STORAGE_MAX_LOCAL_BYTES, + MultiStageQueryContext.getIntermediateSuperSorterStorageMaxLocalBytes( + task.getQuerySpec().getQuery().context() + ) + ).put( + MSQWarnings.CTX_MAX_PARSE_EXCEPTIONS_ALLOWED, + maxParseExceptions + ); this.workerTaskLauncher = new MSQWorkerTaskLauncher( id(), task.getDataSource(), context, - isDurableStageStorageEnabled, - maxParseExceptions, + taskContextOverridesBuilder.build(), // 10 minutes +- 2 minutes jitter TimeUnit.SECONDS.toMillis(600 + ThreadLocalRandom.current().nextInt(-4, 5) * 30L) ); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java index 956c1b74e31a..3bbba26d7509 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java @@ -22,6 +22,7 @@ import com.google.common.base.Function; import com.google.common.base.Preconditions; import com.google.common.base.Suppliers; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import com.google.common.util.concurrent.FutureCallback; @@ -32,6 +33,7 @@ import it.unimi.dsi.fastutil.bytes.ByteArrays; import org.apache.druid.frame.allocation.ArenaMemoryAllocator; import org.apache.druid.frame.channel.BlockingQueueFrameChannel; +import org.apache.druid.frame.channel.ByteTracker; import org.apache.druid.frame.channel.ReadableFileFrameChannel; import org.apache.druid.frame.channel.ReadableFrameChannel; import org.apache.druid.frame.channel.ReadableNilFrameChannel; @@ -41,6 +43,7 @@ import org.apache.druid.frame.key.ClusterByPartitions; import org.apache.druid.frame.processor.BlockingQueueOutputChannelFactory; import org.apache.druid.frame.processor.Bouncer; +import org.apache.druid.frame.processor.ComposingOutputChannelFactory; import org.apache.druid.frame.processor.DurableStorageOutputChannelFactory; import org.apache.druid.frame.processor.FileOutputChannelFactory; import org.apache.druid.frame.processor.FrameChannelMuxer; @@ -161,6 +164,7 @@ public class WorkerImpl implements Worker private final ConcurrentHashMap> stageOutputs = new ConcurrentHashMap<>(); private final ConcurrentHashMap stageCounters = new ConcurrentHashMap<>(); private final ConcurrentHashMap stageKernelMap = new ConcurrentHashMap<>(); + private final ByteTracker intermediateSuperSorterStorageTracker; private final boolean durableStageStorageEnabled; /** @@ -185,6 +189,9 @@ public WorkerImpl(MSQWorkerTask task, WorkerContext context) this.context = context; this.selfDruidNode = context.selfNode(); this.processorBouncer = context.processorBouncer(); + this.intermediateSuperSorterStorageTracker = new ByteTracker( + MultiStageQueryContext.getIntermediateSuperSorterStorageMaxLocalBytes(QueryContext.of(task.getContext())) + ); this.durableStageStorageEnabled = MultiStageQueryContext.isDurableShuffleStorageEnabled( QueryContext.of(task.getContext()) ); @@ -648,8 +655,32 @@ private OutputChannelFactory makeSuperSorterIntermediateOutputChannelFactory( ) { final int frameSize = frameContext.memoryParameters().getLargeFrameSize(); + final File fileChannelDirectory = + new File(tmpDir, StringUtils.format("intermediate_output_stage_%06d", stageNumber)); + final FileOutputChannelFactory fileOutputChannelFactory = + new FileOutputChannelFactory(fileChannelDirectory, frameSize, intermediateSuperSorterStorageTracker); - if (durableStageStorageEnabled) { + if (MultiStageQueryContext.isComposedIntermediateSuperSorterStorageEnabled(QueryContext.of(task.getContext()))) { + if (durableStageStorageEnabled) { + return new ComposingOutputChannelFactory( + ImmutableList.of( + fileOutputChannelFactory, + DurableStorageOutputChannelFactory.createStandardImplementation( + task.getControllerTaskId(), + task().getWorkerNumber(), + stageNumber, + task().getId(), + frameSize, + MSQTasks.makeStorageConnector(context.injector()), + tmpDir + ) + ), + frameSize + ); + } else { + return fileOutputChannelFactory; + } + } else if (durableStageStorageEnabled) { return DurableStorageOutputChannelFactory.createStandardImplementation( task.getControllerTaskId(), task().getWorkerNumber(), @@ -660,9 +691,7 @@ private OutputChannelFactory makeSuperSorterIntermediateOutputChannelFactory( tmpDir ); } else { - final File fileChannelDirectory = - new File(tmpDir, StringUtils.format("intermediate_output_stage_%06d", stageNumber)); - return new FileOutputChannelFactory(fileChannelDirectory, frameSize, null); + return fileOutputChannelFactory; } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQWorkerTaskLauncher.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQWorkerTaskLauncher.java index 3ca45c7419fb..15649fdee6f1 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQWorkerTaskLauncher.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQWorkerTaskLauncher.java @@ -37,13 +37,10 @@ import org.apache.druid.msq.exec.ControllerImpl; import org.apache.druid.msq.exec.WorkerManagerClient; import org.apache.druid.msq.indexing.error.MSQException; -import org.apache.druid.msq.indexing.error.MSQWarnings; import org.apache.druid.msq.indexing.error.TaskStartTimeoutFault; import org.apache.druid.msq.indexing.error.UnknownFault; import org.apache.druid.msq.indexing.error.WorkerFailedFault; -import org.apache.druid.msq.util.MultiStageQueryContext; -import javax.annotation.Nullable; import java.time.Duration; import java.util.ArrayList; import java.util.HashMap; @@ -83,10 +80,6 @@ private enum State private final ControllerContext context; private final ExecutorService exec; private final long maxTaskStartDelayMillis; - private final boolean durableStageStorageEnabled; - - @Nullable - private final Long maxParseExceptions; // Mutable state meant to be accessible by threads outside the main loop. private final SettableFuture stopFuture = SettableFuture.create(); @@ -116,23 +109,23 @@ private enum State // Set of tasks which are issued a cancel request by the controller. private final Set canceledWorkerTasks = ConcurrentHashMap.newKeySet(); + private final Map taskContextOverrides; + public MSQWorkerTaskLauncher( final String controllerTaskId, final String dataSource, final ControllerContext context, - final boolean durableStageStorageEnabled, - @Nullable final Long maxParseExceptions, + final Map taskContextOverrides, final long maxTaskStartDelayMillis ) { this.controllerTaskId = controllerTaskId; this.dataSource = dataSource; this.context = context; + this.taskContextOverrides = taskContextOverrides; this.exec = Execs.singleThreaded( "multi-stage-query-task-launcher[" + StringUtils.encodeForFormat(controllerTaskId) + "]-%s" ); - this.durableStageStorageEnabled = durableStageStorageEnabled; - this.maxParseExceptions = maxParseExceptions; this.maxTaskStartDelayMillis = maxTaskStartDelayMillis; } @@ -317,12 +310,11 @@ private void runNewTasks() { final Map taskContext = new HashMap<>(); - if (durableStageStorageEnabled) { - taskContext.put(MultiStageQueryContext.CTX_ENABLE_DURABLE_SHUFFLE_STORAGE, true); - } - - if (maxParseExceptions != null) { - taskContext.put(MSQWarnings.CTX_MAX_PARSE_EXCEPTIONS_ALLOWED, maxParseExceptions); + for (Map.Entry taskContextOverride : taskContextOverrides.entrySet()) { + if (taskContextOverride.getKey() == null || taskContextOverride.getValue() == null) { + continue; + } + taskContext.put(taskContextOverride.getKey(), taskContextOverride.getValue()); } final int firstTask; diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MultiStageQueryContext.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MultiStageQueryContext.java index 15c80e3caf2d..fce94aa0d5d6 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MultiStageQueryContext.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MultiStageQueryContext.java @@ -59,9 +59,15 @@ public class MultiStageQueryContext private static final boolean DEFAULT_FINALIZE_AGGREGATIONS = true; public static final String CTX_ENABLE_DURABLE_SHUFFLE_STORAGE = "durableShuffleStorage"; + public static final String CTX_COMPOSED_INTERMEDIATE_SUPER_SORTER_STORAGE = + "composedIntermediateSuperSorterStorageEnabled"; + public static final String CTX_INTERMEDIATE_SUPER_SORTER_STORAGE_MAX_LOCAL_BYTES = + "intermediateSuperSorterStorageMaxLocalBytes"; public static final String CTX_CLUSTER_STATISTICS_MERGE_MODE = "clusterStatisticsMergeMode"; public static final String DEFAULT_CLUSTER_STATISTICS_MERGE_MODE = ClusterStatisticsMergeMode.AUTO.toString(); private static final boolean DEFAULT_ENABLE_DURABLE_SHUFFLE_STORAGE = false; + private static final boolean DEFAULT_COMPOSED_INTERMEDIATE_SUPER_SORTER_STORAGE = false; + private static final long DEFAULT_INTERMEDIATE_SUPER_SORTER_STORAGE_MAX_LOCAL_BYTES = Long.MAX_VALUE; public static final String CTX_DESTINATION = "destination"; private static final String DEFAULT_DESTINATION = null; @@ -96,6 +102,22 @@ public static boolean isDurableShuffleStorageEnabled(final QueryContext queryCon ); } + public static boolean isComposedIntermediateSuperSorterStorageEnabled(final QueryContext queryContext) + { + return queryContext.getBoolean( + CTX_COMPOSED_INTERMEDIATE_SUPER_SORTER_STORAGE, + DEFAULT_COMPOSED_INTERMEDIATE_SUPER_SORTER_STORAGE + ); + } + + public static long getIntermediateSuperSorterStorageMaxLocalBytes(final QueryContext queryContext) + { + return queryContext.getLong( + CTX_INTERMEDIATE_SUPER_SORTER_STORAGE_MAX_LOCAL_BYTES, + DEFAULT_INTERMEDIATE_SUPER_SORTER_STORAGE_MAX_LOCAL_BYTES + ); + } + public static ClusterStatisticsMergeMode getClusterStatisticsMergeMode(QueryContext queryContext) { return ClusterStatisticsMergeMode.valueOf( diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQTasksTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQTasksTest.java index 7792712c6a5e..f5885d8f5264 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQTasksTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQTasksTest.java @@ -19,6 +19,7 @@ package org.apache.druid.msq.exec; +import com.google.common.collect.ImmutableMap; import com.google.errorprone.annotations.concurrent.GuardedBy; import org.apache.druid.indexer.TaskLocation; import org.apache.druid.indexer.TaskState; @@ -150,8 +151,7 @@ public void test_queryWithoutEnoughSlots_shouldThrowException() CONTROLLER_ID, "foo", controllerContext, - false, - -1L, + ImmutableMap.of(), TimeUnit.SECONDS.toMillis(5) ); diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java index 19b85034ff93..c6a0b3638039 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java @@ -205,6 +205,8 @@ public class MSQTestBase extends BaseCalciteQueryTest public static final Map DEFAULT_MSQ_CONTEXT = ImmutableMap.builder() .put(MultiStageQueryContext.CTX_ENABLE_DURABLE_SHUFFLE_STORAGE, true) + .put(MultiStageQueryContext.CTX_COMPOSED_INTERMEDIATE_SUPER_SORTER_STORAGE, true) + .put(MultiStageQueryContext.CTX_INTERMEDIATE_SUPER_SORTER_STORAGE_MAX_LOCAL_BYTES, 1) // added so that practically everything still goes to durable storage channel .put(QueryContexts.CTX_SQL_QUERY_ID, "test-query") .put(QueryContexts.FINALIZE_KEY, true) .build(); From fddcb0b1820bae281e4d6946a82eaaa0436e9b79 Mon Sep 17 00:00:00 2001 From: Rohan Garg Date: Tue, 10 Jan 2023 11:08:41 +0530 Subject: [PATCH 12/15] more javadocs --- .../druid/frame/channel/BoundedWritableFrameChannel.java | 4 ++++ .../java/org/apache/druid/frame/channel/ByteTracker.java | 6 ++++-- .../druid/frame/channel/ComposingReadableFrameChannel.java | 4 ++++ .../druid/frame/channel/ComposingWritableFrameChannel.java | 4 ++++ .../frame/processor/ComposingOutputChannelFactory.java | 4 +++- 5 files changed, 19 insertions(+), 3 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/frame/channel/BoundedWritableFrameChannel.java b/processing/src/main/java/org/apache/druid/frame/channel/BoundedWritableFrameChannel.java index 251d9a6b0a48..fff878b9b1c9 100644 --- a/processing/src/main/java/org/apache/druid/frame/channel/BoundedWritableFrameChannel.java +++ b/processing/src/main/java/org/apache/druid/frame/channel/BoundedWritableFrameChannel.java @@ -24,6 +24,10 @@ import javax.annotation.Nullable; import java.io.IOException; +/** + * A writable channel which only allows writing frames till some defined limit. If that limit is breached, every write + * to this channel will throw a {@link org.apache.druid.query.ResourceLimitExceededException}. + */ public class BoundedWritableFrameChannel implements WritableFrameChannel { private final WritableFrameChannel delegate; diff --git a/processing/src/main/java/org/apache/druid/frame/channel/ByteTracker.java b/processing/src/main/java/org/apache/druid/frame/channel/ByteTracker.java index ee42329b126c..4a4828fb1a44 100644 --- a/processing/src/main/java/org/apache/druid/frame/channel/ByteTracker.java +++ b/processing/src/main/java/org/apache/druid/frame/channel/ByteTracker.java @@ -23,10 +23,12 @@ import com.google.errorprone.annotations.concurrent.GuardedBy; import org.apache.druid.query.ResourceLimitExceededException; - +/** + * Tracks the byte usage with an upper bound bytes limit. Reservaction of bytes beyond limit throws + * {@link ResourceLimitExceededException}. + */ public class ByteTracker { - @GuardedBy("this") private final long maxBytes; @GuardedBy("this") diff --git a/processing/src/main/java/org/apache/druid/frame/channel/ComposingReadableFrameChannel.java b/processing/src/main/java/org/apache/druid/frame/channel/ComposingReadableFrameChannel.java index 47b5b4f13a7f..614c021569e6 100644 --- a/processing/src/main/java/org/apache/druid/frame/channel/ComposingReadableFrameChannel.java +++ b/processing/src/main/java/org/apache/druid/frame/channel/ComposingReadableFrameChannel.java @@ -32,6 +32,10 @@ import java.util.Map; import java.util.function.Supplier; +/** + * A composed readable channel to read frames. The channel can encapsulate multiple readable channels in it and + * automatically switches to next channels once the currently read channel is finished. + */ public class ComposingReadableFrameChannel implements ReadableFrameChannel { private final List> channels; diff --git a/processing/src/main/java/org/apache/druid/frame/channel/ComposingWritableFrameChannel.java b/processing/src/main/java/org/apache/druid/frame/channel/ComposingWritableFrameChannel.java index 6abf3ab5dad9..d0872602d744 100644 --- a/processing/src/main/java/org/apache/druid/frame/channel/ComposingWritableFrameChannel.java +++ b/processing/src/main/java/org/apache/druid/frame/channel/ComposingWritableFrameChannel.java @@ -32,6 +32,10 @@ import java.util.Map; import java.util.function.Supplier; +/** + * A composed writable channel to write frames. The channel can encapsulate multiple writable channels in it and + * automatically switches to next channels once the current write channel cannot allow more writes. + */ public class ComposingWritableFrameChannel implements WritableFrameChannel { private final List> channels; diff --git a/processing/src/main/java/org/apache/druid/frame/processor/ComposingOutputChannelFactory.java b/processing/src/main/java/org/apache/druid/frame/processor/ComposingOutputChannelFactory.java index 8d4d8b6c20ab..95604be1d409 100644 --- a/processing/src/main/java/org/apache/druid/frame/processor/ComposingOutputChannelFactory.java +++ b/processing/src/main/java/org/apache/druid/frame/processor/ComposingOutputChannelFactory.java @@ -40,7 +40,9 @@ import java.util.function.Supplier; /** - * + * A channel factory which provides ordered composed channels. The factory encapsulates multiple output channel factories + * and automatically switches between then when the current factory is 'full' for writes. The reads can also encapsulate + * multiple readable channels with automatic switching. */ public class ComposingOutputChannelFactory implements OutputChannelFactory { From 5bfe45763a16d78c6c75372530ebe8edfa6c497d Mon Sep 17 00:00:00 2001 From: Rohan Garg Date: Wed, 18 Jan 2023 17:50:27 +0530 Subject: [PATCH 13/15] Fix up byte calculation --- .../org/apache/druid/msq/exec/WorkerImpl.java | 4 +- .../msq/querykit/BroadcastJoinHelperTest.java | 3 +- .../apache/druid/msq/test/MSQTestBase.java | 4 +- .../java/org/apache/druid/frame/Frame.java | 6 +- .../channel/BoundedWritableFrameChannel.java | 66 ------------------- .../druid/frame/channel/ByteTracker.java | 27 +++++++- .../apache/druid/frame/file/FrameFile.java | 3 +- .../druid/frame/file/FrameFileWriter.java | 28 ++++++-- .../ComposingOutputChannelFactory.java | 2 +- .../DurableStorageOutputChannelFactory.java | 9 ++- .../processor/FileOutputChannelFactory.java | 13 ++-- .../org/apache/druid/frame/FrameTest.java | 7 +- .../druid/frame/file/FrameFileWriterTest.java | 3 +- .../ComposingOutputChannelFactoryTest.java | 2 +- .../processor/FrameProcessorExecutorTest.java | 7 +- .../frame/processor/SuperSorterTest.java | 2 +- .../druid/frame/testutil/FrameTestUtil.java | 7 +- 17 files changed, 89 insertions(+), 104 deletions(-) delete mode 100644 processing/src/main/java/org/apache/druid/frame/channel/BoundedWritableFrameChannel.java diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java index b0008e1bf6d4..ba65ec1c88e6 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java @@ -193,7 +193,7 @@ public WorkerImpl(MSQWorkerTask task, WorkerContext context) this.intermediateSuperSorterStorageTracker = new ByteTracker( MultiStageQueryContext.getIntermediateSuperSorterStorageMaxLocalBytes(QueryContext.of(task.getContext())) ); - this.durableStageStorageEnabled = MultiStageQueryContext.isDurableShuffleStorageEnabled( + this.durableStageStorageEnabled = MultiStageQueryContext.isDurableStorageEnabled( QueryContext.of(task.getContext()) ); } @@ -483,7 +483,7 @@ public InputStream readChannel( if (channel instanceof ReadableNilFrameChannel) { // Build an empty frame file. final ByteArrayOutputStream baos = new ByteArrayOutputStream(); - FrameFileWriter.open(Channels.newChannel(baos), null).close(); + FrameFileWriter.open(Channels.newChannel(baos), null, ByteTracker.unboundedTracker()).close(); final ByteArrayInputStream in = new ByteArrayInputStream(baos.toByteArray()); diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/BroadcastJoinHelperTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/BroadcastJoinHelperTest.java index 149576159a2c..a5bf957a3f2e 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/BroadcastJoinHelperTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/BroadcastJoinHelperTest.java @@ -28,6 +28,7 @@ import org.apache.druid.frame.Frame; import org.apache.druid.frame.FrameType; import org.apache.druid.frame.allocation.ArenaMemoryAllocator; +import org.apache.druid.frame.channel.ByteTracker; import org.apache.druid.frame.channel.ReadableFileFrameChannel; import org.apache.druid.frame.channel.ReadableFrameChannel; import org.apache.druid.frame.file.FrameFile; @@ -212,7 +213,7 @@ public void testBuildTableMemoryLimit() throws IOException sideStageChannelNumberMap.put(0, 0); final List channels = new ArrayList<>(); - channels.add(new ReadableFileFrameChannel(FrameFile.open(testDataFile1, null))); + channels.add(new ReadableFileFrameChannel(FrameFile.open(testDataFile1, ByteTracker.unboundedTracker()))); final List channelReaders = new ArrayList<>(); channelReaders.add(frameReader1); diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java index 37a48a53e76d..7739bfea7dca 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java @@ -226,7 +226,7 @@ public class MSQTestBase extends BaseCalciteQueryTest .putAll(DEFAULT_MSQ_CONTEXT) .put(MultiStageQueryContext.CTX_DURABLE_SHUFFLE_STORAGE, true) .put(MultiStageQueryContext.CTX_COMPOSED_INTERMEDIATE_SUPER_SORTER_STORAGE, true) - .put(MultiStageQueryContext.CTX_INTERMEDIATE_SUPER_SORTER_STORAGE_MAX_LOCAL_BYTES, 100_000_000) // added so that practically everything still goes to durable storage channel + .put(MultiStageQueryContext.CTX_INTERMEDIATE_SUPER_SORTER_STORAGE_MAX_LOCAL_BYTES, 100) // added so that practically everything still goes to durable storage channel .build(); @@ -235,7 +235,7 @@ public class MSQTestBase extends BaseCalciteQueryTest .putAll(DEFAULT_MSQ_CONTEXT) .put(MultiStageQueryContext.CTX_FAULT_TOLERANCE, true) .put(MultiStageQueryContext.CTX_COMPOSED_INTERMEDIATE_SUPER_SORTER_STORAGE, true) - .put(MultiStageQueryContext.CTX_INTERMEDIATE_SUPER_SORTER_STORAGE_MAX_LOCAL_BYTES, 100_000_000) // added so that practically everything still goes to durable storage channel + .put(MultiStageQueryContext.CTX_INTERMEDIATE_SUPER_SORTER_STORAGE_MAX_LOCAL_BYTES, 100) // added so that practically everything still goes to durable storage channel .build(); public static final Map SEQUENTIAL_MERGE_MSQ_CONTEXT = diff --git a/processing/src/main/java/org/apache/druid/frame/Frame.java b/processing/src/main/java/org/apache/druid/frame/Frame.java index 76e10fee0ff9..8185190a28da 100644 --- a/processing/src/main/java/org/apache/druid/frame/Frame.java +++ b/processing/src/main/java/org/apache/druid/frame/Frame.java @@ -25,6 +25,7 @@ import net.jpountz.lz4.LZ4SafeDecompressor; import org.apache.datasketches.memory.Memory; import org.apache.datasketches.memory.WritableMemory; +import org.apache.druid.frame.channel.ByteTracker; import org.apache.druid.io.Channels; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; @@ -357,7 +358,8 @@ public WritableMemory writableMemory() public long writeTo( final WritableByteChannel channel, final boolean compress, - @Nullable final ByteBuffer compressionBuffer + @Nullable final ByteBuffer compressionBuffer, + ByteTracker byteTracker ) throws IOException { if (compress) { @@ -408,10 +410,12 @@ public long writeTo( .xxHash64(0, COMPRESSED_FRAME_HEADER_SIZE + compressedFrameLength, CHECKSUM_SEED); compressionBuffer.putLong(COMPRESSED_FRAME_HEADER_SIZE + compressedFrameLength, checksum); + byteTracker.reserve(compressionBuffer.remaining()); Channels.writeFully(channel, compressionBuffer); return COMPRESSED_FRAME_ENVELOPE_SIZE + compressedFrameLength; } else { + byteTracker.reserve(numBytes); memory.writeTo(0, numBytes, channel); return numBytes; } diff --git a/processing/src/main/java/org/apache/druid/frame/channel/BoundedWritableFrameChannel.java b/processing/src/main/java/org/apache/druid/frame/channel/BoundedWritableFrameChannel.java deleted file mode 100644 index fff878b9b1c9..000000000000 --- a/processing/src/main/java/org/apache/druid/frame/channel/BoundedWritableFrameChannel.java +++ /dev/null @@ -1,66 +0,0 @@ -/* - * 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.frame.channel; - -import com.google.common.util.concurrent.ListenableFuture; - -import javax.annotation.Nullable; -import java.io.IOException; - -/** - * A writable channel which only allows writing frames till some defined limit. If that limit is breached, every write - * to this channel will throw a {@link org.apache.druid.query.ResourceLimitExceededException}. - */ -public class BoundedWritableFrameChannel implements WritableFrameChannel -{ - private final WritableFrameChannel delegate; - private final ByteTracker byteTracker; - - public BoundedWritableFrameChannel(WritableFrameChannel delegate, ByteTracker byteTracker) - { - this.delegate = delegate; - this.byteTracker = byteTracker; - } - - @Override - public void write(FrameWithPartition frameWithPartition) throws IOException - { - byteTracker.reserve(frameWithPartition.frame().numBytes()); - delegate.write(frameWithPartition); - } - - @Override - public void fail(@Nullable Throwable cause) throws IOException - { - delegate.fail(cause); - } - - @Override - public void close() throws IOException - { - delegate.close(); - } - - @Override - public ListenableFuture writabilityFuture() - { - return delegate.writabilityFuture(); - } -} diff --git a/processing/src/main/java/org/apache/druid/frame/channel/ByteTracker.java b/processing/src/main/java/org/apache/druid/frame/channel/ByteTracker.java index 4a4828fb1a44..41eebcc8a635 100644 --- a/processing/src/main/java/org/apache/druid/frame/channel/ByteTracker.java +++ b/processing/src/main/java/org/apache/druid/frame/channel/ByteTracker.java @@ -21,10 +21,11 @@ import com.google.common.base.Preconditions; import com.google.errorprone.annotations.concurrent.GuardedBy; +import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.query.ResourceLimitExceededException; /** - * Tracks the byte usage with an upper bound bytes limit. Reservaction of bytes beyond limit throws + * Tracks the byte usage with an upper bound bytes limit. Reservation of bytes beyond limit throws * {@link ResourceLimitExceededException}. */ public class ByteTracker @@ -42,8 +43,16 @@ public ByteTracker(long maxBytes) public synchronized void reserve(long byteCount) throws ResourceLimitExceededException { Preconditions.checkState(byteCount >= 0, "Can't reserve negative bytes"); + if (currentBytes + byteCount > maxBytes) { - throw new ResourceLimitExceededException(""); + throw new ResourceLimitExceededException( + StringUtils.format( + "Can't allocate any more bytes. maxBytes = %d, currentBytes = %d, requestedBytes = %d", + maxBytes, + currentBytes, + byteCount + ) + ); } currentBytes += byteCount; } @@ -51,7 +60,19 @@ public synchronized void reserve(long byteCount) throws ResourceLimitExceededExc public synchronized void release(long byteCount) { Preconditions.checkState(byteCount >= 0, "Can't release negative bytes"); - Preconditions.checkState(currentBytes >= byteCount, "Can't release more than used bytes"); + Preconditions.checkState( + currentBytes >= byteCount, + StringUtils.format( + "Can't release more than used bytes. currentBytes : %d, releasingBytes : %d", + currentBytes, + byteCount + ) + ); currentBytes -= byteCount; } + + public static ByteTracker unboundedTracker() + { + return new ByteTracker(Long.MAX_VALUE); + } } diff --git a/processing/src/main/java/org/apache/druid/frame/file/FrameFile.java b/processing/src/main/java/org/apache/druid/frame/file/FrameFile.java index da79d35acaa8..37bbe5bf6337 100644 --- a/processing/src/main/java/org/apache/druid/frame/file/FrameFile.java +++ b/processing/src/main/java/org/apache/druid/frame/file/FrameFile.java @@ -246,7 +246,8 @@ static FrameFile open( log.warn("Could not delete frame file [%s]", file); } if (byteTracker != null) { - byteTracker.release(fileLength); + // only release the bytes taken by frames, we don't track the header and footer as of now + byteTracker.release(fileLength - footerLength - FrameFileWriter.MAGIC.length); } }); } diff --git a/processing/src/main/java/org/apache/druid/frame/file/FrameFileWriter.java b/processing/src/main/java/org/apache/druid/frame/file/FrameFileWriter.java index 362e4c8fea60..f855641571ec 100644 --- a/processing/src/main/java/org/apache/druid/frame/file/FrameFileWriter.java +++ b/processing/src/main/java/org/apache/druid/frame/file/FrameFileWriter.java @@ -25,6 +25,7 @@ import org.apache.druid.frame.allocation.AppendableMemory; import org.apache.druid.frame.allocation.HeapMemoryAllocator; import org.apache.druid.frame.allocation.MemoryRange; +import org.apache.druid.frame.channel.ByteTracker; import org.apache.druid.io.Channels; import org.apache.druid.java.util.common.ISE; @@ -50,7 +51,9 @@ public class FrameFileWriter implements Closeable private final WritableByteChannel channel; private final AppendableMemory tableOfContents; private final AppendableMemory partitions; + private final ByteTracker byteTracker; private long bytesWritten = 0; + private long trackedBytes = 0; private int numFrames = 0; private boolean usePartitions = true; private ByteBuffer compressionBuffer; @@ -60,24 +63,30 @@ private FrameFileWriter( final WritableByteChannel channel, @Nullable final ByteBuffer compressionBuffer, final AppendableMemory tableOfContents, - final AppendableMemory partitions + final AppendableMemory partitions, + ByteTracker byteTracker ) { this.channel = channel; this.compressionBuffer = compressionBuffer; this.tableOfContents = tableOfContents; this.partitions = partitions; + this.byteTracker = byteTracker; } /** * Opens a writer for a particular channel. - * - * @param channel destination channel + * @param channel destination channel * @param compressionBuffer result of {@link Frame#compressionBufferSize} for the largest possible frame size that * will be written to this file, or null to allocate buffers dynamically. * Providing an explicit buffer here, if possible, improves performance. + * @param byteTracker tracker to limit the number of bytes that can be written to the frame file */ - public static FrameFileWriter open(final WritableByteChannel channel, @Nullable final ByteBuffer compressionBuffer) + public static FrameFileWriter open( + final WritableByteChannel channel, + @Nullable final ByteBuffer compressionBuffer, + ByteTracker byteTracker + ) { // Unlimited allocator is for convenience. Only a few bytes per frame will be allocated. final HeapMemoryAllocator allocator = HeapMemoryAllocator.unlimited(); @@ -85,7 +94,8 @@ public static FrameFileWriter open(final WritableByteChannel channel, @Nullable channel, compressionBuffer, AppendableMemory.create(allocator), - AppendableMemory.create(allocator) + AppendableMemory.create(allocator), + byteTracker ); } @@ -117,9 +127,13 @@ public void writeFrame(final Frame frame, final int partition) throws IOExceptio writeMagicIfNeeded(); + byteTracker.reserve(1); + trackedBytes++; Channels.writeFully(channel, ByteBuffer.wrap(new byte[]{MARKER_FRAME})); bytesWritten++; - bytesWritten += frame.writeTo(channel, true, getCompressionBuffer(frame.numBytes())); + long frameWrittenBytes = frame.writeTo(channel, true, getCompressionBuffer(frame.numBytes()), byteTracker); + bytesWritten += frameWrittenBytes; + trackedBytes += frameWrittenBytes; // Write *end* of frame to tableOfContents. final MemoryRange tocCursor = tableOfContents.cursor(); @@ -166,6 +180,7 @@ public void abort() throws IOException channel.close(); compressionBuffer = null; closed = true; + byteTracker.release(trackedBytes); } } @@ -206,6 +221,7 @@ public void close() throws IOException // Write footer to the channel. Channels.writeFully(channel, footerBuf); + bytesWritten += footerMemory.getCapacity(); channel.close(); compressionBuffer = null; closed = true; diff --git a/processing/src/main/java/org/apache/druid/frame/processor/ComposingOutputChannelFactory.java b/processing/src/main/java/org/apache/druid/frame/processor/ComposingOutputChannelFactory.java index 95604be1d409..e2bd1e7b7e75 100644 --- a/processing/src/main/java/org/apache/druid/frame/processor/ComposingOutputChannelFactory.java +++ b/processing/src/main/java/org/apache/druid/frame/processor/ComposingOutputChannelFactory.java @@ -56,7 +56,7 @@ public ComposingOutputChannelFactory(List channelFactories } @Override - public OutputChannel openChannel(int partitionNumber) throws IOException + public OutputChannel openChannel(int partitionNumber) { ImmutableList.Builder> writableFrameChannelSuppliersBuilder = ImmutableList.builder(); ImmutableList.Builder> readableFrameChannelSuppliersBuilder = ImmutableList.builder(); diff --git a/processing/src/main/java/org/apache/druid/frame/processor/DurableStorageOutputChannelFactory.java b/processing/src/main/java/org/apache/druid/frame/processor/DurableStorageOutputChannelFactory.java index 2ef32b666af3..b691710ded18 100644 --- a/processing/src/main/java/org/apache/druid/frame/processor/DurableStorageOutputChannelFactory.java +++ b/processing/src/main/java/org/apache/druid/frame/processor/DurableStorageOutputChannelFactory.java @@ -26,6 +26,7 @@ import org.apache.datasketches.memory.Memory; import org.apache.druid.frame.Frame; import org.apache.druid.frame.allocation.ArenaMemoryAllocator; +import org.apache.druid.frame.channel.ByteTracker; import org.apache.druid.frame.channel.DurableStoragePartitionedReadableFrameChannel; import org.apache.druid.frame.channel.ReadableInputStreamFrameChannel; import org.apache.druid.frame.channel.WritableFrameFileChannel; @@ -128,7 +129,8 @@ public OutputChannel openChannel(int partitionNumber) throws IOException new WritableFrameFileChannel( FrameFileWriter.open( Channels.newChannel(storageConnector.write(fileName)), - null + null, + ByteTracker.unboundedTracker() ) ); @@ -178,7 +180,8 @@ public PartitionedOutputChannel openPartitionedChannel(String name, boolean dele new WritableFrameFileChannel( FrameFileWriter.open( Channels.newChannel(countingOutputStream), - ByteBuffer.allocate(Frame.compressionBufferSize(frameSize)) + ByteBuffer.allocate(Frame.compressionBufferSize(frameSize)), + ByteTracker.unboundedTracker() ) ); @@ -250,7 +253,7 @@ public OutputChannel openNilChannel(int 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 { - FrameFileWriter.open(Channels.newChannel(storageConnector.write(fileName)), null).close(); + FrameFileWriter.open(Channels.newChannel(storageConnector.write(fileName)), null, ByteTracker.unboundedTracker()).close(); return OutputChannel.nil(partitionNumber); } catch (IOException e) { diff --git a/processing/src/main/java/org/apache/druid/frame/processor/FileOutputChannelFactory.java b/processing/src/main/java/org/apache/druid/frame/processor/FileOutputChannelFactory.java index 8a7514c0bb7c..93578e972e2f 100644 --- a/processing/src/main/java/org/apache/druid/frame/processor/FileOutputChannelFactory.java +++ b/processing/src/main/java/org/apache/druid/frame/processor/FileOutputChannelFactory.java @@ -22,7 +22,6 @@ import com.google.common.base.Suppliers; import org.apache.druid.frame.Frame; import org.apache.druid.frame.allocation.ArenaMemoryAllocator; -import org.apache.druid.frame.channel.BoundedWritableFrameChannel; import org.apache.druid.frame.channel.ByteTracker; import org.apache.druid.frame.channel.PartitionedReadableFrameChannel; import org.apache.druid.frame.channel.ReadableFileFrameChannel; @@ -72,7 +71,6 @@ public OutputChannel openChannel(int partitionNumber) throws IOException final File file = new File(fileChannelsDirectory, fileName); final WritableFrameChannel writableChannel = - new BoundedWritableFrameChannel( new WritableFrameFileChannel( FrameFileWriter.open( Files.newByteChannel( @@ -80,10 +78,9 @@ public OutputChannel openChannel(int partitionNumber) throws IOException StandardOpenOption.CREATE_NEW, StandardOpenOption.WRITE ), - ByteBuffer.allocate(Frame.compressionBufferSize(frameSize)) + ByteBuffer.allocate(Frame.compressionBufferSize(frameSize)), + byteTracker ) - ), - byteTracker ); final Supplier readableChannelSupplier = Suppliers.memoize( @@ -112,7 +109,6 @@ public PartitionedOutputChannel openPartitionedChannel(String name, boolean dele FileUtils.mkdirp(fileChannelsDirectory); final File file = new File(fileChannelsDirectory, name); WritableFrameChannel writableFrameFileChannel = - new BoundedWritableFrameChannel( new WritableFrameFileChannel( FrameFileWriter.open( Files.newByteChannel( @@ -120,10 +116,9 @@ public PartitionedOutputChannel openPartitionedChannel(String name, boolean dele StandardOpenOption.CREATE_NEW, StandardOpenOption.WRITE ), - ByteBuffer.allocate(Frame.compressionBufferSize(frameSize)) + ByteBuffer.allocate(Frame.compressionBufferSize(frameSize)), + byteTracker ) - ), - byteTracker ); Supplier frameFileSupplier = Suppliers.memoize( () -> { diff --git a/processing/src/test/java/org/apache/druid/frame/FrameTest.java b/processing/src/test/java/org/apache/druid/frame/FrameTest.java index b7745de08794..3439466a2b9d 100644 --- a/processing/src/test/java/org/apache/druid/frame/FrameTest.java +++ b/processing/src/test/java/org/apache/druid/frame/FrameTest.java @@ -25,6 +25,7 @@ import com.google.common.primitives.Ints; import org.apache.datasketches.memory.Memory; import org.apache.datasketches.memory.WritableMemory; +import org.apache.druid.frame.channel.ByteTracker; import org.apache.druid.frame.key.SortColumn; import org.apache.druid.frame.testutil.FrameSequenceBuilder; import org.apache.druid.java.util.common.ByteBufferUtils; @@ -351,7 +352,8 @@ public void testWrapAndWrite() throws Exception frame.writeTo( Channels.newChannel(baos), compressed, - ByteBuffer.allocate(Frame.compressionBufferSize((int) frame.numBytes())) + ByteBuffer.allocate(Frame.compressionBufferSize((int) frame.numBytes())), + ByteTracker.unboundedTracker() ); if (!compressed) { @@ -414,7 +416,8 @@ private static byte[] frameToByteArray(final Frame frame, final boolean compress frame.writeTo( Channels.newChannel(baos), compressed, - ByteBuffer.allocate(Frame.compressionBufferSize((int) frame.numBytes())) + ByteBuffer.allocate(Frame.compressionBufferSize((int) frame.numBytes())), + ByteTracker.unboundedTracker() ); return baos.toByteArray(); } diff --git a/processing/src/test/java/org/apache/druid/frame/file/FrameFileWriterTest.java b/processing/src/test/java/org/apache/druid/frame/file/FrameFileWriterTest.java index dbc8e484de18..5f0e019712be 100644 --- a/processing/src/test/java/org/apache/druid/frame/file/FrameFileWriterTest.java +++ b/processing/src/test/java/org/apache/druid/frame/file/FrameFileWriterTest.java @@ -22,6 +22,7 @@ import org.apache.druid.frame.Frame; import org.apache.druid.frame.FrameType; import org.apache.druid.frame.allocation.ArenaMemoryAllocator; +import org.apache.druid.frame.channel.ByteTracker; import org.apache.druid.frame.testutil.FrameSequenceBuilder; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.segment.TestIndex; @@ -57,7 +58,7 @@ public void test_abort_afterAllFrames() throws IOException final FrameFileWriter fileWriter = FrameFileWriter.open(Files.newByteChannel( file.toPath(), StandardOpenOption.WRITE - ), null); + ), null, ByteTracker.unboundedTracker()); frames.forEach(frame -> { try { diff --git a/processing/src/test/java/org/apache/druid/frame/processor/ComposingOutputChannelFactoryTest.java b/processing/src/test/java/org/apache/druid/frame/processor/ComposingOutputChannelFactoryTest.java index ebe5d4f7537a..9112fe1bed03 100644 --- a/processing/src/test/java/org/apache/druid/frame/processor/ComposingOutputChannelFactoryTest.java +++ b/processing/src/test/java/org/apache/druid/frame/processor/ComposingOutputChannelFactoryTest.java @@ -107,7 +107,7 @@ private static class ThrowingOutputChannelFactory implements OutputChannelFactor { @Override - public OutputChannel openChannel(int partitionNumber) throws IOException + public OutputChannel openChannel(int partitionNumber) { throw new UnsupportedOperationException(); } diff --git a/processing/src/test/java/org/apache/druid/frame/processor/FrameProcessorExecutorTest.java b/processing/src/test/java/org/apache/druid/frame/processor/FrameProcessorExecutorTest.java index 81afa21dcbdf..71b968bf9fa7 100644 --- a/processing/src/test/java/org/apache/druid/frame/processor/FrameProcessorExecutorTest.java +++ b/processing/src/test/java/org/apache/druid/frame/processor/FrameProcessorExecutorTest.java @@ -28,6 +28,7 @@ import org.apache.druid.frame.FrameType; import org.apache.druid.frame.allocation.ArenaMemoryAllocator; import org.apache.druid.frame.channel.BlockingQueueFrameChannel; +import org.apache.druid.frame.channel.ByteTracker; import org.apache.druid.frame.channel.ReadableFileFrameChannel; import org.apache.druid.frame.channel.ReadableFrameChannel; import org.apache.druid.frame.channel.WritableFrameFileChannel; @@ -136,7 +137,8 @@ public void test_runFully() throws Exception new WritableFrameFileChannel( FrameFileWriter.open( Channels.newChannel(Files.newOutputStream(outFile.toPath())), - null + null, + ByteTracker.unboundedTracker() ) ) ); @@ -439,7 +441,8 @@ List writeToNFiles(final StorageAdapter adapter, final int numFiles) throw writers.add( FrameFileWriter.open( Channels.newChannel(Files.newOutputStream(files.get(i).toPath())), - null + null, + ByteTracker.unboundedTracker() ) ); } diff --git a/processing/src/test/java/org/apache/druid/frame/processor/SuperSorterTest.java b/processing/src/test/java/org/apache/druid/frame/processor/SuperSorterTest.java index ac8db8bdef37..6763375cb2d0 100644 --- a/processing/src/test/java/org/apache/druid/frame/processor/SuperSorterTest.java +++ b/processing/src/test/java/org/apache/druid/frame/processor/SuperSorterTest.java @@ -696,7 +696,7 @@ private static List makeFileChannels( final File file = new File(tmpDir, StringUtils.format("channel-%d", i)); files.add(file); writableChannels.add( - new WritableFrameFileChannel(FrameFileWriter.open(Channels.newChannel(new FileOutputStream(file)), null)) + new WritableFrameFileChannel(FrameFileWriter.open(Channels.newChannel(new FileOutputStream(file)), null, ByteTracker.unboundedTracker())) ); } diff --git a/processing/src/test/java/org/apache/druid/frame/testutil/FrameTestUtil.java b/processing/src/test/java/org/apache/druid/frame/testutil/FrameTestUtil.java index 945b9254cd00..593131c2a309 100644 --- a/processing/src/test/java/org/apache/druid/frame/testutil/FrameTestUtil.java +++ b/processing/src/test/java/org/apache/druid/frame/testutil/FrameTestUtil.java @@ -24,6 +24,7 @@ import org.apache.druid.frame.Frame; import org.apache.druid.frame.FrameType; import org.apache.druid.frame.allocation.HeapMemoryAllocator; +import org.apache.druid.frame.channel.ByteTracker; import org.apache.druid.frame.channel.FrameChannelSequence; import org.apache.druid.frame.channel.ReadableFrameChannel; import org.apache.druid.frame.file.FrameFileWriter; @@ -75,7 +76,9 @@ private FrameTestUtil() public static File writeFrameFile(final Sequence frames, final File file) throws IOException { - try (final FrameFileWriter writer = FrameFileWriter.open(Channels.newChannel(new FileOutputStream(file)), null)) { + try (final FrameFileWriter writer = FrameFileWriter.open( + Channels.newChannel(new FileOutputStream(file)), null, ByteTracker.unboundedTracker() + )) { frames.forEach( frame -> { try { @@ -96,7 +99,7 @@ public static File writeFrameFileWithPartitions( final File file ) throws IOException { - try (final FrameFileWriter writer = FrameFileWriter.open(Channels.newChannel(new FileOutputStream(file)), null)) { + try (final FrameFileWriter writer = FrameFileWriter.open(Channels.newChannel(new FileOutputStream(file)), null, ByteTracker.unboundedTracker())) { framesWithPartitions.forEach( frameWithPartition -> { try { From cc96a2b33c719345481685a1636d103ba24d7ebd Mon Sep 17 00:00:00 2001 From: Rohan Garg Date: Thu, 19 Jan 2023 12:47:00 +0530 Subject: [PATCH 14/15] Review comments --- .../main/java/org/apache/druid/msq/exec/WorkerImpl.java | 6 +++--- .../java/org/apache/druid/frame/channel/ByteTracker.java | 6 +++--- .../frame/channel/ComposingReadableFrameChannel.java | 9 ++++----- .../frame/channel/ComposingWritableFrameChannel.java | 4 ++++ .../frame/processor/ComposingOutputChannelFactory.java | 8 ++++++-- .../druid/frame/processor/FileOutputChannelFactory.java | 5 +++-- .../org/apache/druid/frame/testutil/FrameTestUtil.java | 9 ++++++--- 7 files changed, 29 insertions(+), 18 deletions(-) diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java index ba65ec1c88e6..f7c85b164bbd 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/WorkerImpl.java @@ -165,7 +165,7 @@ public class WorkerImpl implements Worker private final ConcurrentHashMap> stageOutputs = new ConcurrentHashMap<>(); private final ConcurrentHashMap stageCounters = new ConcurrentHashMap<>(); private final ConcurrentHashMap stageKernelMap = new ConcurrentHashMap<>(); - private final ByteTracker intermediateSuperSorterStorageTracker; + private final ByteTracker intermediateSuperSorterLocalStorageTracker; private final boolean durableStageStorageEnabled; /** @@ -190,7 +190,7 @@ public WorkerImpl(MSQWorkerTask task, WorkerContext context) this.context = context; this.selfDruidNode = context.selfNode(); this.processorBouncer = context.processorBouncer(); - this.intermediateSuperSorterStorageTracker = new ByteTracker( + this.intermediateSuperSorterLocalStorageTracker = new ByteTracker( MultiStageQueryContext.getIntermediateSuperSorterStorageMaxLocalBytes(QueryContext.of(task.getContext())) ); this.durableStageStorageEnabled = MultiStageQueryContext.isDurableStorageEnabled( @@ -689,7 +689,7 @@ private OutputChannelFactory makeSuperSorterIntermediateOutputChannelFactory( final File fileChannelDirectory = new File(tmpDir, StringUtils.format("intermediate_output_stage_%06d", stageNumber)); final FileOutputChannelFactory fileOutputChannelFactory = - new FileOutputChannelFactory(fileChannelDirectory, frameSize, intermediateSuperSorterStorageTracker); + new FileOutputChannelFactory(fileChannelDirectory, frameSize, intermediateSuperSorterLocalStorageTracker); if (MultiStageQueryContext.isComposedIntermediateSuperSorterStorageEnabled(QueryContext.of(task.getContext()))) { if (durableStageStorageEnabled) { diff --git a/processing/src/main/java/org/apache/druid/frame/channel/ByteTracker.java b/processing/src/main/java/org/apache/druid/frame/channel/ByteTracker.java index 41eebcc8a635..9dbe2e024bba 100644 --- a/processing/src/main/java/org/apache/druid/frame/channel/ByteTracker.java +++ b/processing/src/main/java/org/apache/druid/frame/channel/ByteTracker.java @@ -44,7 +44,7 @@ public synchronized void reserve(long byteCount) throws ResourceLimitExceededExc { Preconditions.checkState(byteCount >= 0, "Can't reserve negative bytes"); - if (currentBytes + byteCount > maxBytes) { + if (Math.addExact(currentBytes, byteCount) > maxBytes) { throw new ResourceLimitExceededException( StringUtils.format( "Can't allocate any more bytes. maxBytes = %d, currentBytes = %d, requestedBytes = %d", @@ -54,7 +54,7 @@ public synchronized void reserve(long byteCount) throws ResourceLimitExceededExc ) ); } - currentBytes += byteCount; + currentBytes = Math.addExact(currentBytes, byteCount); } public synchronized void release(long byteCount) @@ -68,7 +68,7 @@ public synchronized void release(long byteCount) byteCount ) ); - currentBytes -= byteCount; + currentBytes = Math.subtractExact(currentBytes, byteCount); } public static ByteTracker unboundedTracker() diff --git a/processing/src/main/java/org/apache/druid/frame/channel/ComposingReadableFrameChannel.java b/processing/src/main/java/org/apache/druid/frame/channel/ComposingReadableFrameChannel.java index 614c021569e6..db43ee249a11 100644 --- a/processing/src/main/java/org/apache/druid/frame/channel/ComposingReadableFrameChannel.java +++ b/processing/src/main/java/org/apache/druid/frame/channel/ComposingReadableFrameChannel.java @@ -25,17 +25,18 @@ import com.google.common.util.concurrent.ListenableFuture; import org.apache.druid.frame.Frame; +import javax.annotation.concurrent.NotThreadSafe; import java.util.ArrayList; import java.util.Collections; import java.util.HashSet; import java.util.List; -import java.util.Map; import java.util.function.Supplier; /** * A composed readable channel to read frames. The channel can encapsulate multiple readable channels in it and * automatically switches to next channels once the currently read channel is finished. */ +@NotThreadSafe public class ComposingReadableFrameChannel implements ReadableFrameChannel { private final List> channels; @@ -45,16 +46,14 @@ public class ComposingReadableFrameChannel implements ReadableFrameChannel public ComposingReadableFrameChannel( int partition, List> channels, - Map> partitionToChannelMap + HashSet validChannels ) { Preconditions.checkNotNull(channels, "channels is null"); - Preconditions.checkNotNull(partitionToChannelMap, "partitionToChannelMap is null"); - if (partitionToChannelMap.get(partition) == null) { + if (validChannels == null) { // no writes for the partition, send an empty readable channel this.channels = ImmutableList.of(() -> ReadableNilFrameChannel.INSTANCE); } else { - HashSet validChannels = partitionToChannelMap.get(partition); Preconditions.checkState(validChannels.size() > 0, "No channels found for partition " + partition); ImmutableList.Builder> validChannelsBuilder = ImmutableList.builder(); ArrayList sortedChannelIds = new ArrayList<>(validChannels); diff --git a/processing/src/main/java/org/apache/druid/frame/channel/ComposingWritableFrameChannel.java b/processing/src/main/java/org/apache/druid/frame/channel/ComposingWritableFrameChannel.java index d0872602d744..b9ca905ed2bf 100644 --- a/processing/src/main/java/org/apache/druid/frame/channel/ComposingWritableFrameChannel.java +++ b/processing/src/main/java/org/apache/druid/frame/channel/ComposingWritableFrameChannel.java @@ -66,6 +66,10 @@ public void write(FrameWithPartition frameWithPartition) throws IOException .add(currentIndex); } catch (ResourceLimitExceededException rlee) { + // currently we're falling back to next available channel after we receive an RLEE. This is done so that the + // exception is automatically passed up to the user incase all the channels are exhausted. If in future, more + // cases come up to dictate control flow, then we can switch to returning a custom object from the channel's write + // operation. channels.get(currentIndex).get().close(); currentIndex++; if (currentIndex >= channels.size()) { diff --git a/processing/src/main/java/org/apache/druid/frame/processor/ComposingOutputChannelFactory.java b/processing/src/main/java/org/apache/druid/frame/processor/ComposingOutputChannelFactory.java index e2bd1e7b7e75..8ca9aa4f6fa9 100644 --- a/processing/src/main/java/org/apache/druid/frame/processor/ComposingOutputChannelFactory.java +++ b/processing/src/main/java/org/apache/druid/frame/processor/ComposingOutputChannelFactory.java @@ -86,7 +86,7 @@ public OutputChannel openChannel(int partitionNumber) () -> new ComposingReadableFrameChannel( partitionNumber, readableFrameChannelSuppliersBuilder.build(), - partitionToChannelMap + partitionToChannelMap.get(partitionNumber) ) )::get; return OutputChannel.pair( @@ -148,7 +148,11 @@ public ReadableFrameChannel getReadableFrameChannel(int partitionNumber) ); } - return new ComposingReadableFrameChannel(partitionNumber, suppliers.build(), partitionToChannelMap); + return new ComposingReadableFrameChannel( + partitionNumber, + suppliers.build(), + partitionToChannelMap.get(partitionNumber) + ); } @Override diff --git a/processing/src/main/java/org/apache/druid/frame/processor/FileOutputChannelFactory.java b/processing/src/main/java/org/apache/druid/frame/processor/FileOutputChannelFactory.java index 93578e972e2f..6d6950fe4172 100644 --- a/processing/src/main/java/org/apache/druid/frame/processor/FileOutputChannelFactory.java +++ b/processing/src/main/java/org/apache/druid/frame/processor/FileOutputChannelFactory.java @@ -33,6 +33,7 @@ import org.apache.druid.java.util.common.FileUtils; import org.apache.druid.java.util.common.StringUtils; +import javax.annotation.Nullable; import java.io.File; import java.io.IOException; import java.io.UncheckedIOException; @@ -54,12 +55,12 @@ public class FileOutputChannelFactory implements OutputChannelFactory public FileOutputChannelFactory( final File fileChannelsDirectory, final int frameSize, - final ByteTracker byteTracker + @Nullable final ByteTracker byteTracker ) { this.fileChannelsDirectory = fileChannelsDirectory; this.frameSize = frameSize; - this.byteTracker = byteTracker == null ? new ByteTracker(Long.MAX_VALUE) : byteTracker; + this.byteTracker = byteTracker == null ? ByteTracker.unboundedTracker() : byteTracker; } @Override diff --git a/processing/src/test/java/org/apache/druid/frame/testutil/FrameTestUtil.java b/processing/src/test/java/org/apache/druid/frame/testutil/FrameTestUtil.java index 593131c2a309..26fb41d0c30d 100644 --- a/processing/src/test/java/org/apache/druid/frame/testutil/FrameTestUtil.java +++ b/processing/src/test/java/org/apache/druid/frame/testutil/FrameTestUtil.java @@ -76,9 +76,12 @@ private FrameTestUtil() public static File writeFrameFile(final Sequence frames, final File file) throws IOException { - try (final FrameFileWriter writer = FrameFileWriter.open( - Channels.newChannel(new FileOutputStream(file)), null, ByteTracker.unboundedTracker() - )) { + try ( + final FileOutputStream fos = new FileOutputStream(file); + final FrameFileWriter writer = FrameFileWriter.open( + Channels.newChannel(fos), null, ByteTracker.unboundedTracker() + ) + ) { frames.forEach( frame -> { try { From 7b5dadf270be99b196bb741162184fd1293c1d89 Mon Sep 17 00:00:00 2001 From: Rohan Garg Date: Fri, 20 Jan 2023 16:00:11 +0530 Subject: [PATCH 15/15] user documentation for parameters and tracking comment --- docs/multi-stage-query/reference.md | 2 ++ .../main/java/org/apache/druid/frame/file/FrameFile.java | 8 +++++++- website/.spelling | 3 +++ 3 files changed, 12 insertions(+), 1 deletion(-) diff --git a/docs/multi-stage-query/reference.md b/docs/multi-stage-query/reference.md index e8d6b31c398f..86689c10028c 100644 --- a/docs/multi-stage-query/reference.md +++ b/docs/multi-stage-query/reference.md @@ -324,6 +324,8 @@ The following table lists the context parameters for the MSQ task engine: | `clusterStatisticsMergeMode` | Whether to use parallel or sequential mode for merging of the worker sketches. Can be `PARALLEL`, `SEQUENTIAL` or `AUTO`. See [Sketch Merging Mode](#sketch-merging-mode) for more information. | `PARALLEL` | | `durableShuffleStorage` | SELECT, INSERT, REPLACE

Whether to use durable storage for shuffle mesh. To use this feature, configure the durable storage at the server level using `druid.msq.intermediate.storage.enable=true`). If these properties are not configured, any query with the context variable `durableShuffleStorage=true` fails with a configuration error.

| `false` | | `faultTolerance` | SELECT, INSERT, REPLACE

Whether to turn on fault tolerance mode or not. Failed workers are retried based on [Limits](#limits). Cannot be used when `durableShuffleStorage` is explicitly set to false. | `false` | +| `composedIntermediateSuperSorterStorageEnabled` | SELECT, INSERT, REPLACE

Whether to enable automatic fallback to durable storage from local storage for sorting's intermediate data. Requires to setup `intermediateSuperSorterStorageMaxLocalBytes` limit for local storage and durable shuffle storage feature as well.| `false` | +| `intermediateSuperSorterStorageMaxLocalBytes` | SELECT, INSERT, REPLACE

Whether to enable a byte limit on local storage for sorting's intermediate data. If that limit is crossed, the task fails with `ResourceLimitExceededException`.| `9223372036854775807` | ## Sketch Merging Mode This section details the advantages and performance of various Cluster By Statistics Merge Modes. diff --git a/processing/src/main/java/org/apache/druid/frame/file/FrameFile.java b/processing/src/main/java/org/apache/druid/frame/file/FrameFile.java index 37bbe5bf6337..a6fbda9829dd 100644 --- a/processing/src/main/java/org/apache/druid/frame/file/FrameFile.java +++ b/processing/src/main/java/org/apache/druid/frame/file/FrameFile.java @@ -246,7 +246,13 @@ static FrameFile open( log.warn("Could not delete frame file [%s]", file); } if (byteTracker != null) { - // only release the bytes taken by frames, we don't track the header and footer as of now + // Only release the bytes taken by frames, we don't track the header and footer as of now. + // The reason for not tracking them currently is that they are written in the close method of a channel + // incase of empty frame files. To track them, we'd either need to augment close method to pass error objects + // if the storage can't write the header/footer data or create a new method in the channel interface to allow + // for pre-reserving bytes for them before the close method is called. + // For now, they are left untracked also on the assumption that their size would be much smaller than the + // actual frame data. But in future, it would be better to track their sizes as well. byteTracker.release(fileLength - footerLength - FrameFileWriter.MAGIC.length); } }); diff --git a/website/.spelling b/website/.spelling index bd7430c31901..218a68d60aaf 100644 --- a/website/.spelling +++ b/website/.spelling @@ -722,6 +722,9 @@ rowsInMemory segmentSortOrder rowsPerSegment durableShuffleStorage +composedIntermediateSuperSorterStorageEnabled +intermediateSuperSorterStorageMaxLocalBytes +ResourceLimitExceededException # Aggregations groupByEnableMultiValueUnnesting APPROX_COUNT_DISTINCT_DS_HLL