diff --git a/.travis.yml b/.travis.yml
index 003331be2e34..b8511511d3aa 100644
--- a/.travis.yml
+++ b/.travis.yml
@@ -86,7 +86,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/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..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
@@ -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;
@@ -26,10 +27,14 @@
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;
+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 +60,32 @@ 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
+ {
+ 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);
+ }
+
/**
* 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..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
@@ -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));
+ }
+
+ // 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);
+ 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));
+ }
+ }
+
+ // empty 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/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/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 7bf8dffa8ecf..d3257a72fb98 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
@@ -53,6 +53,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;
@@ -153,7 +154,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.PartialKeyStatisticsInformation;
import org.apache.druid.msq.util.DimensionSchemaUtils;
@@ -293,6 +293,9 @@ public ControllerImpl(
{
this.task = task;
this.context = context;
+ this.isDurableStorageEnabled = MultiStageQueryContext.isDurableStorageEnabled(
+ task.getQuerySpec().getQuery().context()
+ );
}
@Override
@@ -587,6 +590,25 @@ private QueryDefinition initializeQueryDefAndState(final Closer closer)
.orElse(MSQWarnings.DEFAULT_MAX_PARSE_EXCEPTIONS_ALLOWED);
}
+ ImmutableMap.Builder taskContextOverridesBuilder = ImmutableMap.builder();
+ taskContextOverridesBuilder
+ .put(
+ MultiStageQueryContext.CTX_DURABLE_SHUFFLE_STORAGE,
+ isDurableStorageEnabled
+ ).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(),
@@ -600,8 +622,7 @@ private QueryDefinition initializeQueryDefAndState(final Closer closer)
}
});
},
- isDurableStorageEnabled,
- 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 b1fa1dfd6aaf..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
@@ -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,8 @@
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;
import org.apache.druid.frame.processor.FrameProcessor;
@@ -50,6 +54,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;
@@ -102,8 +107,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;
@@ -162,6 +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 intermediateSuperSorterLocalStorageTracker;
private final boolean durableStageStorageEnabled;
/**
@@ -186,6 +190,9 @@ public WorkerImpl(MSQWorkerTask task, WorkerContext context)
this.context = context;
this.selfDruidNode = context.selfNode();
this.processorBouncer = context.processorBouncer();
+ this.intermediateSuperSorterLocalStorageTracker = new ByteTracker(
+ MultiStageQueryContext.getIntermediateSuperSorterStorageMaxLocalBytes(QueryContext.of(task.getContext()))
+ );
this.durableStageStorageEnabled = MultiStageQueryContext.isDurableStorageEnabled(
QueryContext.of(task.getContext())
);
@@ -476,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());
@@ -661,13 +668,61 @@ private OutputChannelFactory makeStageOutputChannelFactory(final FrameContext fr
stageNumber,
task().getId(),
frameSize,
- MSQTasks.makeStorageConnector(context.injector())
+ MSQTasks.makeStorageConnector(context.injector()),
+ context.tempDir()
);
} else {
final File fileChannelDirectory =
new File(context.tempDir(), StringUtils.format("output_stage_%06d", stageNumber));
- return new FileOutputChannelFactory(fileChannelDirectory, frameSize);
+ return new FileOutputChannelFactory(fileChannelDirectory, frameSize, null);
+ }
+ }
+
+ private OutputChannelFactory makeSuperSorterIntermediateOutputChannelFactory(
+ final FrameContext frameContext,
+ final int stageNumber,
+ final File tmpDir
+ )
+ {
+ 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, intermediateSuperSorterLocalStorageTracker);
+
+ 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(),
+ stageNumber,
+ task().getId(),
+ frameSize,
+ MSQTasks.makeStorageConnector(context.injector()),
+ tmpDir
+ );
+ } else {
+ return fileOutputChannelFactory;
}
}
@@ -871,16 +926,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()
);
@@ -958,7 +1019,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());
@@ -1078,10 +1140,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
@@ -1123,21 +1185,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..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
@@ -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;
@@ -50,7 +51,22 @@ public OutputChannel openChannel(int partitionNumber) throws IOException
new CountingWritableFrameChannel(
baseChannel.getWritableChannel(),
channelCounters,
- baseChannel.getPartitionNumber()
+ partitionNumber
+ )
+ );
+ }
+
+ @Override
+ public PartitionedOutputChannel openPartitionedChannel(String name, boolean deleteAfterRead) throws IOException
+ {
+ final PartitionedOutputChannel baseChannel = baseFactory.openPartitionedChannel(name, deleteAfterRead);
+
+ return baseChannel.mapWritableChannel(
+ baseWritableChannel ->
+ new CountingWritableFrameChannel(
+ baseChannel.getWritableChannel(),
+ 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 1daf6ef32ef7..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,12 +31,12 @@ public class CountingWritableFrameChannel implements WritableFrameChannel
{
private final WritableFrameChannel baseChannel;
private final ChannelCounters channelCounters;
- private final int partitionNumber;
+ private final Integer partitionNumber;
public CountingWritableFrameChannel(
final WritableFrameChannel baseChannel,
final ChannelCounters channelCounters,
- final int partitionNumber
+ final Integer partitionNumber
)
{
this.baseChannel = baseChannel;
@@ -48,7 +48,7 @@ public CountingWritableFrameChannel(
public void write(FrameWithPartition frame) throws IOException
{
baseChannel.write(frame);
- channelCounters.addFrame(partitionNumber, 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/indexing/DurableStorageCleaner.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/DurableStorageCleaner.java
index 822e3fd460b9..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,6 +24,7 @@
import com.google.common.collect.Sets;
import com.google.inject.Inject;
import com.google.inject.Provider;
+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;
@@ -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/MSQWorkerTaskLauncher.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQWorkerTaskLauncher.java
index 1be409ed6a9f..b6d1665015dc 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
@@ -39,15 +39,12 @@
import org.apache.druid.msq.exec.MSQTasks;
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.TooManyAttemptsForJob;
import org.apache.druid.msq.indexing.error.TooManyAttemptsForWorker;
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;
@@ -89,10 +86,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();
@@ -123,6 +116,7 @@ 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;
// tasks to clean up due to retries
private final Set tasksToCleanup = ConcurrentHashMap.newKeySet();
@@ -138,21 +132,18 @@ public MSQWorkerTaskLauncher(
final String dataSource,
final ControllerContext context,
final RetryTask retryTask,
- 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.retryTask = retryTask;
- this.durableStageStorageEnabled = durableStageStorageEnabled;
- this.maxParseExceptions = maxParseExceptions;
this.maxTaskStartDelayMillis = maxTaskStartDelayMillis;
}
@@ -378,12 +369,11 @@ private void runNewTasks()
{
final Map taskContext = new HashMap<>();
- if (durableStageStorageEnabled) {
- taskContext.put(MultiStageQueryContext.CTX_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/shuffle/DurableStorageInputChannelFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/DurableStorageInputChannelFactory.java
index 0d32a2d6bbbb..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
@@ -23,6 +23,8 @@
import org.apache.commons.io.IOUtils;
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;
@@ -113,7 +115,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/DurableStorageOutputChannelFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/DurableStorageOutputChannelFactory.java
deleted file mode 100644
index 3bc0f72c8924..000000000000
--- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/shuffle/DurableStorageOutputChannelFactory.java
+++ /dev/null
@@ -1,161 +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.msq.shuffle;
-
-import com.google.common.base.Preconditions;
-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.FrameFileWriter;
-import org.apache.druid.frame.processor.OutputChannel;
-import org.apache.druid.frame.processor.OutputChannelFactory;
-import org.apache.druid.java.util.common.ISE;
-import org.apache.druid.java.util.common.logger.Logger;
-import org.apache.druid.storage.StorageConnector;
-
-import java.io.IOException;
-import java.io.OutputStreamWriter;
-import java.nio.channels.Channels;
-import java.nio.charset.StandardCharsets;
-
-public class DurableStorageOutputChannelFactory implements OutputChannelFactory
-{
-
- private static final Logger LOG = new Logger(DurableStorageOutputChannelFactory.class);
-
- private final String controllerTaskId;
- private final int workerNumber;
- private final int stageNumber;
- private final String taskId;
- private final int frameSize;
- private final StorageConnector storageConnector;
-
- public DurableStorageOutputChannelFactory(
- final String controllerTaskId,
- final int workerNumber,
- final int stageNumber,
- final String taskId,
- final int frameSize,
- final StorageConnector storageConnector
- )
- {
- this.controllerTaskId = Preconditions.checkNotNull(controllerTaskId, "controllerTaskId");
- this.workerNumber = workerNumber;
- this.stageNumber = stageNumber;
- this.taskId = taskId;
- this.frameSize = frameSize;
- this.storageConnector = Preconditions.checkNotNull(storageConnector, "storageConnector");
- }
-
- /**
- * Creates an instance that is the standard production implementation. Closeable items are registered with
- * the provided Closer.
- */
- public static DurableStorageOutputChannelFactory createStandardImplementation(
- final String controllerTaskId,
- final int workerNumber,
- final int stageNumber,
- final String taskId,
- final int frameSize,
- final StorageConnector storageConnector
- )
- {
- return new DurableStorageOutputChannelFactory(
- controllerTaskId,
- workerNumber,
- stageNumber,
- taskId,
- frameSize,
- storageConnector
- );
- }
-
- @Override
- public OutputChannel openChannel(int partitionNumber) throws IOException
- {
- final String fileName = DurableStorageUtils.getPartitionOutputsFileNameForPartition(
- controllerTaskId,
- stageNumber,
- workerNumber,
- taskId,
- partitionNumber
- );
- final WritableFrameFileChannel writableChannel =
- new WritableFrameFileChannel(
- FrameFileWriter.open(
- Channels.newChannel(storageConnector.write(fileName)),
- null
- )
- );
-
- return OutputChannel.pair(
- writableChannel,
- ArenaMemoryAllocator.createOnHeap(frameSize),
- () -> ReadableNilFrameChannel.INSTANCE, // remote reads should happen via the DurableStorageInputChannelFactory
- partitionNumber
- );
- }
-
- @Override
- public OutputChannel openNilChannel(int partitionNumber)
- {
- final String fileName = DurableStorageUtils.getPartitionOutputsFileNameForPartition(
- controllerTaskId,
- stageNumber,
- workerNumber,
- taskId,
- partitionNumber
- );
- // As tasks dependent on output of this partition will forever block if no file is present in RemoteStorage. Hence, writing a dummy frame.
- try {
-
- FrameFileWriter.open(Channels.newChannel(storageConnector.write(fileName)), null).close();
- return OutputChannel.nil(partitionNumber);
- }
- catch (IOException e) {
- throw new ISE(
- e,
- "Unable to create empty remote output of stage [%d], partition [%d] for worker [%d]",
- stageNumber,
- partitionNumber,
- workerNumber
- );
- }
- }
-
- /**
- * Creates a file with name __success and adds the worker's id which has successfully written its outputs. While reading
- * this file can be used to find out the worker which has written its outputs completely.
- * Rename operation is not very quick in cloud storage like S3 due to which this alternative
- * route has been taken.
- * If the success file is already present in the location, then this method is a noop
- */
- public void createSuccessFile(String taskId) throws IOException
- {
- String fileName = DurableStorageUtils.getSuccessFilePath(controllerTaskId, stageNumber, workerNumber);
- if (storageConnector.pathExists(fileName)) {
- LOG.warn("Path [%s] already exists. Won't attempt to rewrite on top of it.", fileName);
- return;
- }
- OutputStreamWriter os = new OutputStreamWriter(storageConnector.write(fileName), StandardCharsets.UTF_8);
- os.write(taskId); // Add some dummy content in the file
- os.close();
- }
-}
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 a596879fad50..9f69b1cd8e94 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
@@ -67,6 +67,13 @@ public class MultiStageQueryContext
public static final String CTX_CLUSTER_STATISTICS_MERGE_MODE = "clusterStatisticsMergeMode";
public static final String DEFAULT_CLUSTER_STATISTICS_MERGE_MODE = ClusterStatisticsMergeMode.PARALLEL.toString();
+ public static final String CTX_INTERMEDIATE_SUPER_SORTER_STORAGE_MAX_LOCAL_BYTES =
+ "intermediateSuperSorterStorageMaxLocalBytes";
+ public static final String CTX_COMPOSED_INTERMEDIATE_SUPER_SORTER_STORAGE =
+ "composedIntermediateSuperSorterStorageEnabled";
+ 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;
@@ -108,6 +115,22 @@ public static boolean isFaultToleranceEnabled(final QueryContext queryContext)
);
}
+ 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/MSQSelectTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java
index d56afd774680..f52be84e7d0f 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.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;
@@ -32,7 +33,6 @@
import org.apache.druid.msq.indexing.MSQSpec;
import org.apache.druid.msq.indexing.MSQTuningConfig;
import org.apache.druid.msq.indexing.error.CannotParseExternalDataFault;
-import org.apache.druid.msq.shuffle.DurableStorageUtils;
import org.apache.druid.msq.test.CounterSnapshotBuilder;
import org.apache.druid.msq.test.MSQTestBase;
import org.apache.druid.msq.test.MSQTestFileUtils;
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 cc087092103e..bb1d1b1dbc56 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;
@@ -171,8 +172,7 @@ public void test_queryWithoutEnoughSlots_shouldThrowException()
"foo",
controllerContext,
(task, fault) -> {},
- false,
- -1L,
+ ImmutableMap.of(),
TimeUnit.SECONDS.toMillis(5)
);
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..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,11 +21,11 @@
import com.google.common.base.Optional;
import com.google.common.collect.ImmutableList;
+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;
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/querykit/BroadcastJoinHelperTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/querykit/BroadcastJoinHelperTest.java
index 4a32144d2745..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;
@@ -119,8 +120,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 +213,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, 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 5ecf17f2ab07..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
@@ -224,13 +224,19 @@ public class MSQTestBase extends BaseCalciteQueryTest
public static final Map DURABLE_STORAGE_MSQ_CONTEXT =
ImmutableMap.builder()
.putAll(DEFAULT_MSQ_CONTEXT)
- .put(MultiStageQueryContext.CTX_DURABLE_SHUFFLE_STORAGE, true).build();
+ .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) // added so that practically everything still goes to durable storage channel
+ .build();
public static final Map FAULT_TOLERANCE_MSQ_CONTEXT =
ImmutableMap.builder()
.putAll(DEFAULT_MSQ_CONTEXT)
- .put(MultiStageQueryContext.CTX_FAULT_TOLERANCE, true).build();
+ .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) // added so that practically everything still goes to durable storage channel
+ .build();
public static final Map SEQUENTIAL_MERGE_MSQ_CONTEXT =
ImmutableMap.builder()
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 347bc47c5c49..bfb6981c6d5d 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
@@ -53,13 +53,13 @@
public class MultiStageQueryContextTest
{
@Test
- public void isDurableStorageEnabled_noParameterSetReturnsDefaultValue()
+ public void isDurableShuffleStorageEnabled_noParameterSetReturnsDefaultValue()
{
Assert.assertFalse(MultiStageQueryContext.isDurableStorageEnabled(QueryContext.empty()));
}
@Test
- public void isDurableStorageEnabled_parameterSetReturnsCorrectValue()
+ public void isDurableShuffleStorageEnabled_parameterSetReturnsCorrectValue()
{
Map propertyMap = ImmutableMap.of(CTX_DURABLE_SHUFFLE_STORAGE, "true");
Assert.assertTrue(MultiStageQueryContext.isDurableStorageEnabled(QueryContext.of(propertyMap)));
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..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
@@ -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;
@@ -62,9 +63,28 @@ public boolean pathExists(String path)
@Override
public InputStream read(String path) throws IOException
+ {
+ return buildInputStream(new GetObjectRequest(config.getBucket(), objectPath(path)));
+ }
+
+ @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 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)),
+ getObjectRequest,
new ObjectOpenFunction()
{
@Override
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..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
@@ -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";
+
+ // 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);
+ 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);
+ }
+ }
+
+ // empty 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/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/ByteTracker.java b/processing/src/main/java/org/apache/druid/frame/channel/ByteTracker.java
new file mode 100644
index 000000000000..9dbe2e024bba
--- /dev/null
+++ b/processing/src/main/java/org/apache/druid/frame/channel/ByteTracker.java
@@ -0,0 +1,78 @@
+/*
+ * 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.java.util.common.StringUtils;
+import org.apache.druid.query.ResourceLimitExceededException;
+
+/**
+ * Tracks the byte usage with an upper bound bytes limit. Reservation of bytes beyond limit throws
+ * {@link ResourceLimitExceededException}.
+ */
+public class ByteTracker
+{
+ 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 (Math.addExact(currentBytes, byteCount) > maxBytes) {
+ throw new ResourceLimitExceededException(
+ StringUtils.format(
+ "Can't allocate any more bytes. maxBytes = %d, currentBytes = %d, requestedBytes = %d",
+ maxBytes,
+ currentBytes,
+ byteCount
+ )
+ );
+ }
+ currentBytes = Math.addExact(currentBytes, byteCount);
+ }
+
+ public synchronized void release(long byteCount)
+ {
+ Preconditions.checkState(byteCount >= 0, "Can't release negative bytes");
+ Preconditions.checkState(
+ currentBytes >= byteCount,
+ StringUtils.format(
+ "Can't release more than used bytes. currentBytes : %d, releasingBytes : %d",
+ currentBytes,
+ byteCount
+ )
+ );
+ currentBytes = Math.subtractExact(currentBytes, byteCount);
+ }
+
+ public static ByteTracker unboundedTracker()
+ {
+ return new ByteTracker(Long.MAX_VALUE);
+ }
+}
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..db43ee249a11
--- /dev/null
+++ b/processing/src/main/java/org/apache/druid/frame/channel/ComposingReadableFrameChannel.java
@@ -0,0 +1,146 @@
+/*
+ * 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 javax.annotation.concurrent.NotThreadSafe;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+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;
+ private ReadableFrameChannel currentChannel;
+ private int currentIndex;
+
+ public ComposingReadableFrameChannel(
+ int partition,
+ List> channels,
+ HashSet validChannels
+ )
+ {
+ Preconditions.checkNotNull(channels, "channels is null");
+ if (validChannels == null) {
+ // no writes for the partition, send an empty readable channel
+ this.channels = ImmutableList.of(() -> ReadableNilFrameChannel.INSTANCE);
+ } else {
+ 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..b9ca905ed2bf
--- /dev/null
+++ b/processing/src/main/java/org/apache/druid/frame/channel/ComposingWritableFrameChannel.java
@@ -0,0 +1,103 @@
+/*
+ * 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;
+
+/**
+ * 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;
+ 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) {
+ // 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()) {
+ 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/channel/DurableStoragePartitionedReadableFrameChannel.java b/processing/src/main/java/org/apache/druid/frame/channel/DurableStoragePartitionedReadableFrameChannel.java
new file mode 100644
index 000000000000..0e0e6680df37
--- /dev/null
+++ b/processing/src/main/java/org/apache/druid/frame/channel/DurableStoragePartitionedReadableFrameChannel.java
@@ -0,0 +1,93 @@
+/*
+ * 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 getReadableFrameChannel(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 = 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),
+ frameFileFullPath,
+ remoteInputStreamPool,
+ true
+ );
+ }
+ catch (IOException e) {
+ throw new UncheckedIOException(e);
+ }
+ }
+
+ @Override
+ public void close()
+ {
+ try {
+ storageConnector.deleteFile(frameFileFullPath);
+ footerFile.delete();
+ }
+ catch (IOException e) {
+ throw new UncheckedIOException(e);
+ }
+ }
+}
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..4edd904afda3
--- /dev/null
+++ b/processing/src/main/java/org/apache/druid/frame/channel/PartitionedReadableFrameChannel.java
@@ -0,0 +1,44 @@
+/*
+ * 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;
+import java.io.IOException;
+
+/**
+ * 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 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/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..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
@@ -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;
@@ -84,10 +85,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 +115,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;
@@ -145,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);
}
/**
@@ -159,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));
@@ -198,11 +202,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 +234,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();
@@ -261,6 +245,16 @@ 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) {
+ // 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);
+ }
});
}
@@ -270,11 +264,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 +287,7 @@ static FrameFile open(final File file, final int maxMmapSize, final Flag... flag
*/
public int numFrames()
{
- return numFrames;
+ return frameFileFooter.getNumFrames();
}
/**
@@ -303,7 +295,7 @@ public int numFrames()
*/
public int numPartitions()
{
- return numPartitions;
+ return frameFileFooter.getNumPartitions();
}
/**
@@ -313,21 +305,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 +315,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 +355,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 +390,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..3d2bd29f3423
--- /dev/null
+++ b/processing/src/main/java/org/apache/druid/frame/file/FrameFileFooter.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.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.
+ * The schema for footer is as described by {@link FrameFile}.
+ */
+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/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/BlockingQueueOutputChannelFactory.java b/processing/src/main/java/org/apache/druid/frame/processor/BlockingQueueOutputChannelFactory.java
index 94c24b3220da..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
@@ -46,6 +46,12 @@ public OutputChannel openChannel(final int partitionNumber)
);
}
+ @Override
+ public PartitionedOutputChannel openPartitionedChannel(String name, boolean deleteAfterRead)
+ {
+ 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/ComposingOutputChannelFactory.java b/processing/src/main/java/org/apache/druid/frame/processor/ComposingOutputChannelFactory.java
new file mode 100644
index 000000000000..8ca9aa4f6fa9
--- /dev/null
+++ b/processing/src/main/java/org/apache/druid/frame/processor/ComposingOutputChannelFactory.java
@@ -0,0 +1,183 @@
+/*
+ * 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;
+
+/**
+ * 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
+{
+ 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)
+ {
+ 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());
+ }
+
+ // 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(),
+ partitionToChannelMap
+ );
+ Supplier readableFrameChannelSupplier = Suppliers.memoize(
+ () -> new ComposingReadableFrameChannel(
+ partitionNumber,
+ readableFrameChannelSuppliersBuilder.build(),
+ partitionToChannelMap.get(partitionNumber)
+ )
+ )::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());
+ }
+ // 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(),
+ partitionToChannelMap
+ );
+
+ List> readableFrameChannelSuppliers =
+ 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
+ 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.get(partitionNumber)
+ );
+ }
+
+ @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)
+ {
+ // 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/main/java/org/apache/druid/frame/processor/DurableStorageOutputChannelFactory.java b/processing/src/main/java/org/apache/druid/frame/processor/DurableStorageOutputChannelFactory.java
new file mode 100644
index 000000000000..b691710ded18
--- /dev/null
+++ b/processing/src/main/java/org/apache/druid/frame/processor/DurableStorageOutputChannelFactory.java
@@ -0,0 +1,288 @@
+/*
+ * 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.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.ByteTracker;
+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.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;
+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;
+ private final int workerNumber;
+ private final int stageNumber;
+ 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,
+ final int workerNumber,
+ final int stageNumber,
+ final String taskId,
+ final int frameSize,
+ final StorageConnector storageConnector,
+ final File tmpDir
+ )
+ {
+ this.controllerTaskId = Preconditions.checkNotNull(controllerTaskId, "controllerTaskId");
+ this.workerNumber = workerNumber;
+ this.stageNumber = stageNumber;
+ 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"));
+ }
+
+ /**
+ * Creates an instance that is the standard production implementation. Closeable items are registered with
+ * the provided Closer.
+ */
+ public static DurableStorageOutputChannelFactory createStandardImplementation(
+ final String controllerTaskId,
+ final int workerNumber,
+ final int stageNumber,
+ final String taskId,
+ final int frameSize,
+ final StorageConnector storageConnector,
+ final File tmpDir
+ )
+ {
+ return new DurableStorageOutputChannelFactory(
+ controllerTaskId,
+ workerNumber,
+ stageNumber,
+ taskId,
+ frameSize,
+ storageConnector,
+ tmpDir
+ );
+ }
+
+ @Override
+ public OutputChannel openChannel(int partitionNumber) throws IOException
+ {
+ final String fileName = DurableStorageUtils.getPartitionOutputsFileNameForPartition(
+ controllerTaskId,
+ stageNumber,
+ workerNumber,
+ taskId,
+ partitionNumber
+ );
+ final WritableFrameFileChannel writableChannel =
+ new WritableFrameFileChannel(
+ FrameFileWriter.open(
+ Channels.newChannel(storageConnector.write(fileName)),
+ null,
+ ByteTracker.unboundedTracker()
+ )
+ );
+
+ return OutputChannel.pair(
+ writableChannel,
+ ArenaMemoryAllocator.createOnHeap(frameSize),
+ () -> {
+ 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 openPartitionedChannel(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)),
+ ByteTracker.unboundedTracker()
+ )
+ );
+
+ final Supplier channelSizeSupplier = countingOutputStream::getCount;
+
+ final File footerFile = new File(tmpDir, fileName + "_footer");
+ // build supplier for reading 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
+ 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)) {
+ 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)
+ {
+ final String fileName = DurableStorageUtils.getPartitionOutputsFileNameForPartition(
+ controllerTaskId,
+ stageNumber,
+ workerNumber,
+ taskId,
+ partitionNumber
+ );
+ // As tasks dependent on output of this partition will forever block if no file is present in RemoteStorage. Hence, writing a dummy frame.
+ try {
+
+ FrameFileWriter.open(Channels.newChannel(storageConnector.write(fileName)), null, ByteTracker.unboundedTracker()).close();
+ return OutputChannel.nil(partitionNumber);
+ }
+ catch (IOException e) {
+ throw new ISE(
+ e,
+ "Unable to create empty remote output of stage [%d], partition [%d] for worker [%d]",
+ stageNumber,
+ partitionNumber,
+ workerNumber
+ );
+ }
+ }
+
+ /**
+ * Creates a file with name __success and adds the worker's id which has successfully written its outputs. While reading
+ * this file can be used to find out the worker which has written its outputs completely.
+ * Rename operation is not very quick in cloud storage like S3 due to which this alternative
+ * route has been taken.
+ * If the success file is already present in the location, then this method is a noop
+ */
+ public void createSuccessFile(String taskId) throws IOException
+ {
+ String fileName = DurableStorageUtils.getSuccessFilePath(controllerTaskId, stageNumber, workerNumber);
+ if (storageConnector.pathExists(fileName)) {
+ LOG.warn("Path [%s] already exists. Won't attempt to rewrite on top of it.", fileName);
+ return;
+ }
+ OutputStreamWriter os = new OutputStreamWriter(storageConnector.write(fileName), StandardCharsets.UTF_8);
+ os.write(taskId); // Add some dummy content in the file
+ os.close();
+ }
+}
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..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
@@ -22,16 +22,21 @@
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.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;
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;
import java.nio.ByteBuffer;
import java.nio.file.Files;
import java.nio.file.StandardOpenOption;
@@ -45,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,
+ @Nullable final ByteTracker byteTracker
+ )
{
this.fileChannelsDirectory = fileChannelsDirectory;
this.frameSize = frameSize;
+ this.byteTracker = byteTracker == null ? ByteTracker.unboundedTracker() : byteTracker;
}
@Override
@@ -60,22 +71,23 @@ 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 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) {
@@ -92,6 +104,64 @@ public OutputChannel openChannel(int partitionNumber) throws IOException
);
}
+ @Override
+ public PartitionedOutputChannel openPartitionedChannel(String name, boolean deleteAfterRead) throws IOException
+ {
+ FileUtils.mkdirp(fileChannelsDirectory);
+ final File file = new File(fileChannelsDirectory, name);
+ WritableFrameChannel writableFrameFileChannel =
+ 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, byteTracker, FrameFile.Flag.DELETE_ON_CLOSE)
+ : FrameFile.open(file, byteTracker);
+ }
+ catch (IOException e) {
+ throw new UncheckedIOException(e);
+ }
+ }
+ )::get;
+ final Supplier partitionedReadableFrameChannelSupplier = Suppliers.memoize(
+ () -> new PartitionedReadableFrameChannel()
+ {
+ @Override
+ public ReadableFrameChannel getReadableFrameChannel(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/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/main/java/org/apache/druid/frame/processor/OutputChannelFactory.java b/processing/src/main/java/org/apache/druid/frame/processor/OutputChannelFactory.java
index 11de551332e2..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
@@ -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 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/PartitionedOutputChannel.java b/processing/src/main/java/org/apache/druid/frame/processor/PartitionedOutputChannel.java
new file mode 100644
index 000000000000..3e455545b049
--- /dev/null
+++ b/processing/src/main/java/org/apache/druid/frame/processor/PartitionedOutputChannel.java
@@ -0,0 +1,122 @@
+/*
+ * 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;
+
+/**
+ * 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
+ 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..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
@@ -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 = mergerOutputFileName(inLevel, i);
+ PartitionedReadableFrameChannel partitionedReadableFrameChannel =
+ levelAndRankToReadableChannelMap.remove(levelAndRankKey)
+ .getReadableChannelSupplier()
+ .get();
+ in.add(partitionedReadableFrameChannel.getReadableFrameChannel(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(mergerOutputFileName(inLevel, i))
+ .getReadableChannelSupplier()
+ .get()
+ .getReadableFrameChannel(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,
+ final List partitionedReadableChannelsToClose
)
{
try {
final WritableFrameChannel writableChannel;
final MemoryAllocator frameAllocator;
+ String levelAndRankKey = mergerOutputFileName(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.openPartitionedChannel(
+ 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,17 @@ 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) {
+ throw new UncheckedIOException(
+ StringUtils.format("Could not close channel for level [%d] and rank [%d]", level, rank),
+ e
+ );
+ }
+ }
}
});
}
@@ -811,15 +800,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 +825,12 @@ private void cleanUp()
inputChannelsToRead.clear();
}
- private File mergerOutputFile(final int level, final long rank)
+ private String mergerOutputFileName(final int level, final long rank)
{
- return new File(directory, StringUtils.format("merged.%d.%d", level, rank));
+ return StringUtils.format("merged.%d.%d", level, rank);
}
+
/**
* Returns a string encapsulating the current state of this object.
*/
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/util/DurableStorageUtils.java
similarity index 85%
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/util/DurableStorageUtils.java
index df3b86a5c7ea..6e6bbd9fe927 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/util/DurableStorageUtils.java
@@ -17,7 +17,7 @@
* under the License.
*/
-package org.apache.druid.msq.shuffle;
+package org.apache.druid.frame.util;
import org.apache.druid.common.utils.IdUtils;
import org.apache.druid.java.util.common.StringUtils;
@@ -102,4 +102,23 @@ public static String getPartitionOutputsFileNameForPartition(
partitionNumber
);
}
+
+ /**
+ * 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,
+ 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/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/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/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..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 {
@@ -70,7 +71,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/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..9112fe1bed03
--- /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)
+ {
+ 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/FrameProcessorExecutorTest.java b/processing/src/test/java/org/apache/druid/frame/processor/FrameProcessorExecutorTest.java
index 349fdb2efe23..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()
)
)
);
@@ -150,7 +152,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()
);
@@ -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()
)
);
}
@@ -482,7 +485,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/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);
+ }
+}
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..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
@@ -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;
@@ -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, null),
+ isDurableStorage ? new DurableStorageOutputChannelFactory(
+ "0",
+ 0,
+ 0,
+ "0",
+ FRAME_SIZE,
+ new LocalFileStorageConnector(tempFolder),
+ tempFolder
+ ) : new FileOutputChannelFactory(tempFolder, FRAME_SIZE, null),
2,
2,
-1,
@@ -161,6 +182,7 @@ public static class ParameterizedCasesTest extends InitializedNullHandlingTest
private final int maxActiveProcessors;
private final int maxChannelsPerProcessor;
private final int numThreads;
+ private final boolean isComposedStorage;
private StorageAdapter adapter;
private RowSignature signature;
@@ -174,7 +196,8 @@ public ParameterizedCasesTest(
int numChannels,
int maxActiveProcessors,
int maxChannelsPerProcessor,
- int numThreads
+ int numThreads,
+ boolean isComposedStorage
)
{
this.maxRowsPerFrame = maxRowsPerFrame;
@@ -183,6 +206,7 @@ public ParameterizedCasesTest(
this.maxActiveProcessors = maxActiveProcessors;
this.maxChannelsPerProcessor = maxChannelsPerProcessor;
this.numThreads = numThreads;
+ this.isComposedStorage = isComposedStorage;
}
@Parameterized.Parameters(
@@ -191,7 +215,8 @@ public ParameterizedCasesTest(
+ "numChannels = {2}, "
+ "maxActiveProcessors = {3}, "
+ "maxChannelsPerProcessor = {4}, "
- + "numThreads = {5}"
+ + "numThreads = {5}, "
+ + "isComposedStorage = {6}"
)
public static Iterable constructorFeeder()
{
@@ -203,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 isComposedStorage : new boolean[]{true, false}) {
+ if (maxActiveProcessors >= maxChannelsPerProcessor) {
+ constructors.add(
+ new Object[]{
+ maxRowsPerFrame,
+ maxBytesPerFrame,
+ numChannels,
+ maxActiveProcessors,
+ maxChannelsPerProcessor,
+ numThreads,
+ isComposedStorage
+ }
+ );
+ }
}
}
}
@@ -274,6 +302,22 @@ private OutputChannels verifySuperSorter(
final ClusterByPartitions clusterByPartitions
) throws Exception
{
+ final File tempFolder = temporaryFolder.newFolder();
+ 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();
@@ -285,9 +329,8 @@ private OutputChannels verifySuperSorter(
clusterBy,
clusterByPartitionsFuture,
exec,
- temporaryFolder.newFolder(),
- new FileOutputChannelFactory(temporaryFolder.newFolder(), maxBytesPerFrame),
- () -> ArenaMemoryAllocator.createOnHeap(maxBytesPerFrame),
+ new FileOutputChannelFactory(tempFolder, maxBytesPerFrame, null),
+ outputChannelFactory,
maxActiveProcessors,
maxChannelsPerProcessor,
-1,
@@ -594,6 +637,45 @@ public void test_clusterByQualityLongDescRowNumberAsc_fourPartitions() throws Ex
verifySuperSorter(clusterBy, partitions);
}
+ @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);
+ }
+
private RowKey createKey(final ClusterBy clusterBy, final Object... objects)
{
final RowSignature keySignature = KeyTestUtils.createKeySignature(clusterBy.getColumns(), signature);
@@ -614,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()))
);
}
@@ -643,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;
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..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
@@ -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,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)) {
+ try (
+ final FileOutputStream fos = new FileOutputStream(file);
+ final FrameFileWriter writer = FrameFileWriter.open(
+ Channels.newChannel(fos), null, ByteTracker.unboundedTracker()
+ )
+ ) {
frames.forEach(
frame -> {
try {
@@ -96,7 +102,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 {
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