MSQ: Rework memory management. (#17057) (#17210)

This patch reworks memory management to better support multi-threaded
workers running in shared JVMs. There are two main changes.

First, processing buffers and threads are moved from a per-JVM model to
a per-worker model. This enables queries to hold processing buffers
without blocking other concurrently-running queries. Changes:

- Introduce ProcessingBuffersSet and ProcessingBuffers to hold the
  per-worker and per-work-order processing buffers (respectively). On Peons,
  this is the JVM-wide processing pool. On Indexers, this is a per-worker
  pool of on-heap buffers. (This change fixes a bug on Indexers where
  excessive processing buffers could be used if MSQ tasks ran concurrently
  with realtime tasks.)

- Add "bufferPool" argument to GroupingEngine#process so a per-worker pool
  can be passed in.

- Add "druid.msq.task.memory.maxThreads" property, which controls the
  maximum number of processing threads to use per task. This allows usage of
  multiple processing buffers per task if admins desire.

- IndexerWorkerContext acquires processingBuffers when creating the FrameContext
  for a work order, and releases them when closing the FrameContext.

- Add "usesProcessingBuffers()" to FrameProcessorFactory so workers know
  how many sets of processing buffers are needed to run a given query.

Second, adjustments to how WorkerMemoryParameters slices up bundles, to
favor more memory for sorting and segment generation. Changes:

- Instead of using same-sized bundles for processing and for sorting,
  workers now use minimally-sized processing bundles (just enough to read
  inputs plus a little overhead). The rest is devoted to broadcast data
  buffering, sorting, and segment-building.

- Segment-building is now limited to 1 concurrent segment per work order.
  This allows each segment-building action to use more memory. Note that
  segment-building is internally multi-threaded to a degree. (Build and
  persist can run concurrently.)

- Simplify frame size calculations by removing the distinction between
  "standard" and "large" frames. The new default frame size is the same
  as the old "standard" frames, 1 MB. The original goal of of the large
  frames was to reduce the number of temporary files during sorting, but
  I think we can achieve the same thing by simply merging a larger number
  of standard frames at once.

- Remove the small worker adjustment that was added in #14117 to account
  for an extra frame involved in writing to durable storage. Instead,
  account for the extra frame whenever we are actually using durable storage.

- Cap super-sorter parallelism using the number of output partitions, rather
  than using a hard coded cap at 4. Note that in practice, so far, this cap
  has not been relevant for tasks because they have only been using a single
  processing thread anyway.

Co-authored-by: Gian Merlino <gianmerlino@gmail.com>
This commit is contained in:
Kashif Faraz 2024-10-01 19:50:24 +05:30 committed by GitHub
parent 9b192bd5a2
commit 23b9039a02
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
67 changed files with 2016 additions and 890 deletions

View File

@ -378,7 +378,6 @@ public class GroupByTypeInterfaceBenchmark
final GroupingEngine groupingEngine = new GroupingEngine(
druidProcessingConfig,
configSupplier,
bufferPool,
groupByResourcesReservationPool,
TestHelper.makeJsonMapper(),
new ObjectMapper(new SmileFactory()),
@ -387,7 +386,8 @@ public class GroupByTypeInterfaceBenchmark
factory = new GroupByQueryRunnerFactory(
groupingEngine,
new GroupByQueryQueryToolChest(groupingEngine, groupByResourcesReservationPool)
new GroupByQueryQueryToolChest(groupingEngine, groupByResourcesReservationPool),
bufferPool
);
}

View File

@ -362,14 +362,13 @@ public class CachingClusteredClientBenchmark
final GroupingEngine groupingEngine = new GroupingEngine(
processingConfig,
configSupplier,
bufferPool,
groupByResourcesReservationPool,
mapper,
mapper,
QueryRunnerTestHelper.NOOP_QUERYWATCHER
);
final GroupByQueryQueryToolChest toolChest = new GroupByQueryQueryToolChest(groupingEngine, groupByResourcesReservationPool);
return new GroupByQueryRunnerFactory(groupingEngine, toolChest);
return new GroupByQueryRunnerFactory(groupingEngine, toolChest, bufferPool);
}
@TearDown(Level.Trial)

View File

@ -495,7 +495,6 @@ public class GroupByBenchmark
final GroupingEngine groupingEngine = new GroupingEngine(
druidProcessingConfig,
configSupplier,
bufferPool,
groupByResourcesReservationPool,
TestHelper.makeJsonMapper(),
new ObjectMapper(new SmileFactory()),
@ -504,7 +503,8 @@ public class GroupByBenchmark
factory = new GroupByQueryRunnerFactory(
groupingEngine,
new GroupByQueryQueryToolChest(groupingEngine, groupByResourcesReservationPool)
new GroupByQueryQueryToolChest(groupingEngine, groupByResourcesReservationPool),
bufferPool
);
}

View File

@ -99,8 +99,7 @@ public class MapVirtualColumnGroupByTest extends InitializedNullHandlingTest
return 1;
}
},
() -> config,
new StupidPool<>("map-virtual-column-groupby-test", () -> ByteBuffer.allocate(1024)),
GroupByQueryConfig::new,
groupByResourcesReservationPool,
TestHelper.makeJsonMapper(),
new DefaultObjectMapper(),
@ -109,7 +108,8 @@ public class MapVirtualColumnGroupByTest extends InitializedNullHandlingTest
final GroupByQueryRunnerFactory factory = new GroupByQueryRunnerFactory(
groupingEngine,
new GroupByQueryQueryToolChest(groupingEngine, groupByResourcesReservationPool)
new GroupByQueryQueryToolChest(groupingEngine, groupByResourcesReservationPool),
new StupidPool<>("map-virtual-column-groupby-test", () -> ByteBuffer.allocate(1024))
);
runner = QueryRunnerTestHelper.makeQueryRunner(

View File

@ -19,7 +19,6 @@
package org.apache.druid.msq.exec;
import com.google.common.base.Preconditions;
import org.apache.druid.msq.indexing.error.MSQException;
import org.apache.druid.msq.indexing.error.NotEnoughMemoryFault;
import org.apache.druid.msq.kernel.controller.ControllerQueryKernel;
@ -29,10 +28,10 @@ import org.apache.druid.msq.statistics.ClusterByStatisticsCollectorImpl;
* Class for determining how much JVM heap to allocate to various purposes for {@link Controller}.
*
* First, look at how much of total JVM heap that is dedicated for MSQ; see
* {@link MemoryIntrospector#usableMemoryInJvm()}.
* {@link MemoryIntrospector#memoryPerTask()}.
*
* Then, we split up that total amount of memory into equally-sized portions per {@link Controller}; see
* {@link MemoryIntrospector#numQueriesInJvm()}. The number of controllers is based entirely on server configuration,
* {@link MemoryIntrospector#numTasksInJvm()}. The number of controllers is based entirely on server configuration,
* which makes the calculation robust to different queries running simultaneously in the same JVM.
*
* Then, we split that up into a chunk used for input channels, and a chunk used for partition statistics.
@ -70,29 +69,28 @@ public class ControllerMemoryParameters
final int maxWorkerCount
)
{
final long usableMemoryInJvm = memoryIntrospector.usableMemoryInJvm();
final int numControllersInJvm = memoryIntrospector.numQueriesInJvm();
Preconditions.checkArgument(usableMemoryInJvm > 0, "Usable memory[%s] must be > 0", usableMemoryInJvm);
Preconditions.checkArgument(numControllersInJvm > 0, "Number of controllers[%s] must be > 0", numControllersInJvm);
Preconditions.checkArgument(maxWorkerCount > 0, "Number of workers[%s] must be > 0", maxWorkerCount);
final long memoryPerController = usableMemoryInJvm / numControllersInJvm;
final long memoryForInputChannels = WorkerMemoryParameters.memoryNeededForInputChannels(maxWorkerCount);
final long totalMemory = memoryIntrospector.memoryPerTask();
final long memoryForInputChannels =
WorkerMemoryParameters.computeProcessorMemoryForInputChannels(
maxWorkerCount,
WorkerMemoryParameters.DEFAULT_FRAME_SIZE
);
final int partitionStatisticsMaxRetainedBytes = (int) Math.min(
memoryPerController - memoryForInputChannels,
totalMemory - memoryForInputChannels,
PARTITION_STATS_MAX_MEMORY
);
if (partitionStatisticsMaxRetainedBytes < PARTITION_STATS_MIN_MEMORY) {
final long requiredMemory = memoryForInputChannels + PARTITION_STATS_MIN_MEMORY;
final long requiredTaskMemory = memoryForInputChannels + PARTITION_STATS_MIN_MEMORY;
throw new MSQException(
new NotEnoughMemoryFault(
memoryIntrospector.computeJvmMemoryRequiredForUsableMemory(requiredMemory),
memoryIntrospector.computeJvmMemoryRequiredForTaskMemory(requiredTaskMemory),
memoryIntrospector.totalMemoryInJvm(),
usableMemoryInJvm,
numControllersInJvm,
memoryIntrospector.numProcessorsInJvm(),
0
memoryIntrospector.memoryPerTask(),
memoryIntrospector.numTasksInJvm(),
memoryIntrospector.numProcessingThreads(),
maxWorkerCount,
1
)
);
}

View File

@ -24,7 +24,7 @@ public class Limits
/**
* Maximum number of columns that can appear in a frame signature.
* <p>
* Somewhat less than {@link WorkerMemoryParameters#STANDARD_FRAME_SIZE} divided by typical minimum column size:
* Somewhat less than {@link WorkerMemoryParameters#DEFAULT_FRAME_SIZE} divided by typical minimum column size:
* {@link org.apache.druid.frame.allocation.AppendableMemory#DEFAULT_INITIAL_ALLOCATION_SIZE}.
*/
public static final int MAX_FRAME_COLUMNS = 2000;

View File

@ -19,10 +19,8 @@
package org.apache.druid.msq.exec;
import org.apache.druid.msq.kernel.WorkOrder;
/**
* Introspector used to generate {@link ControllerMemoryParameters}.
* Introspector used to generate {@link WorkerMemoryParameters} and {@link ControllerMemoryParameters}.
*/
public interface MemoryIntrospector
{
@ -32,34 +30,23 @@ public interface MemoryIntrospector
long totalMemoryInJvm();
/**
* Amount of memory usable for the multi-stage query engine in the entire JVM.
*
* This may be an expensive operation. For example, the production implementation {@link MemoryIntrospectorImpl}
* estimates size of all lookups as part of computing this value.
* Amount of memory alloted to each {@link Worker} or {@link Controller}.
*/
long usableMemoryInJvm();
long memoryPerTask();
/**
* Amount of total JVM memory required for a particular amount of usable memory to be available.
*
* This may be an expensive operation. For example, the production implementation {@link MemoryIntrospectorImpl}
* estimates size of all lookups as part of computing this value.
* Computes the amount of total JVM memory that would be required for a particular memory allotment per task, i.e.,
* a particular return value from {@link #memoryPerTask()}.
*/
long computeJvmMemoryRequiredForUsableMemory(long usableMemory);
long computeJvmMemoryRequiredForTaskMemory(long memoryPerTask);
/**
* Maximum number of queries that run simultaneously in this JVM.
*
* On workers, this is the maximum number of {@link Worker} that run simultaneously in this JVM. See
* {@link WorkerMemoryParameters} for how memory is divided among and within {@link WorkOrder} run by a worker.
*
* On controllers, this is the maximum number of {@link Controller} that run simultaneously. See
* {@link ControllerMemoryParameters} for how memory is used by controllers.
* Maximum number of tasks ({@link Worker} or {@link Controller}) that run simultaneously in this JVM.
*/
int numQueriesInJvm();
int numTasksInJvm();
/**
* Maximum number of processing threads that can be used at once in this JVM.
* Maximum number of processing threads that can be used at once by each {@link Worker} or {@link Controller}.
*/
int numProcessorsInJvm();
int numProcessingThreads();
}

View File

@ -20,12 +20,14 @@
package org.apache.druid.msq.exec;
import com.google.common.collect.ImmutableList;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.query.DruidProcessingConfig;
import org.apache.druid.query.lookup.LookupExtractor;
import org.apache.druid.query.lookup.LookupExtractorFactoryContainer;
import org.apache.druid.query.lookup.LookupExtractorFactoryContainerProvider;
import javax.annotation.Nullable;
import java.util.List;
/**
@ -34,37 +36,47 @@ import java.util.List;
public class MemoryIntrospectorImpl implements MemoryIntrospector
{
private static final Logger log = new Logger(MemoryIntrospectorImpl.class);
private static final long LOOKUP_FOOTPRINT_INIT = Long.MIN_VALUE;
private final LookupExtractorFactoryContainerProvider lookupProvider;
private final long totalMemoryInJvm;
private final int numQueriesInJvm;
private final int numProcessorsInJvm;
private final double usableMemoryFraction;
private final int numTasksInJvm;
private final int numProcessingThreads;
/**
* Lookup footprint per task, set the first time {@link #memoryPerTask()} is called.
*/
private volatile long lookupFootprint = LOOKUP_FOOTPRINT_INIT;
@Nullable
private final LookupExtractorFactoryContainerProvider lookupProvider;
/**
* Create an introspector.
*
* @param lookupProvider provider of lookups; we use this to subtract lookup size from total JVM memory when
* computing usable memory
* @param totalMemoryInJvm maximum JVM heap memory
* @param usableMemoryFraction fraction of JVM memory, after subtracting lookup overhead, that we consider usable
* for multi-stage queries
* @param numQueriesInJvm maximum number of {@link Controller} or {@link Worker} that may run concurrently
* @param numProcessorsInJvm size of processing thread pool, typically {@link DruidProcessingConfig#getNumThreads()}
* for {@link Controller} or {@link Worker}
* @param numTasksInJvm maximum number of {@link Controller} or {@link Worker} that may run concurrently
* @param numProcessingThreads size of processing thread pool, typically {@link DruidProcessingConfig#getNumThreads()}
* @param lookupProvider provider of lookups; we use this to subtract lookup size from total JVM memory when
* computing usable memory. Ignored if null. This is used once the first time
* {@link #memoryPerTask()} is called, then the footprint is cached. As such, it provides
* a point-in-time view only.
*/
public MemoryIntrospectorImpl(
final LookupExtractorFactoryContainerProvider lookupProvider,
final long totalMemoryInJvm,
final double usableMemoryFraction,
final int numQueriesInJvm,
final int numProcessorsInJvm
final int numTasksInJvm,
final int numProcessingThreads,
@Nullable final LookupExtractorFactoryContainerProvider lookupProvider
)
{
this.lookupProvider = lookupProvider;
this.totalMemoryInJvm = totalMemoryInJvm;
this.numQueriesInJvm = numQueriesInJvm;
this.numProcessorsInJvm = numProcessorsInJvm;
this.usableMemoryFraction = usableMemoryFraction;
this.numTasksInJvm = numTasksInJvm;
this.numProcessingThreads = numProcessingThreads;
this.lookupProvider = lookupProvider;
}
@Override
@ -74,33 +86,52 @@ public class MemoryIntrospectorImpl implements MemoryIntrospector
}
@Override
public long usableMemoryInJvm()
public long memoryPerTask()
{
final long totalMemory = totalMemoryInJvm();
final long totalLookupFootprint = computeTotalLookupFootprint(true);
return Math.max(
0,
(long) ((totalMemory - totalLookupFootprint) * usableMemoryFraction)
(long) ((totalMemoryInJvm - getTotalLookupFootprint()) * usableMemoryFraction) / numTasksInJvm
);
}
@Override
public long computeJvmMemoryRequiredForUsableMemory(long usableMemory)
public long computeJvmMemoryRequiredForTaskMemory(long memoryPerTask)
{
final long totalLookupFootprint = computeTotalLookupFootprint(false);
return (long) Math.ceil(usableMemory / usableMemoryFraction + totalLookupFootprint);
if (memoryPerTask <= 0) {
throw new IAE("Invalid memoryPerTask[%d], expected a positive number", memoryPerTask);
}
return (long) Math.ceil(memoryPerTask * numTasksInJvm / usableMemoryFraction) + getTotalLookupFootprint();
}
@Override
public int numQueriesInJvm()
public int numTasksInJvm()
{
return numQueriesInJvm;
return numTasksInJvm;
}
@Override
public int numProcessorsInJvm()
public int numProcessingThreads()
{
return numProcessorsInJvm;
return numProcessingThreads;
}
/**
* Get a possibly-cached value of {@link #computeTotalLookupFootprint()}. The underlying computation method is
* called just once, meaning this is not a good way to track the size of lookups over time. This is done to keep
* memory calculations as consistent as possible.
*/
private long getTotalLookupFootprint()
{
if (lookupFootprint == LOOKUP_FOOTPRINT_INIT) {
synchronized (this) {
if (lookupFootprint == LOOKUP_FOOTPRINT_INIT) {
lookupFootprint = computeTotalLookupFootprint();
}
}
}
return lookupFootprint;
}
/**
@ -108,11 +139,13 @@ public class MemoryIntrospectorImpl implements MemoryIntrospector
*
* Correctness of this approach depends on lookups being loaded *before* calling this method. Luckily, this is the
* typical mode of operation, since by default druid.lookup.enableLookupSyncOnStartup = true.
*
* @param logFootprint whether footprint should be logged
*/
private long computeTotalLookupFootprint(final boolean logFootprint)
private long computeTotalLookupFootprint()
{
if (lookupProvider == null) {
return 0;
}
final List<String> lookupNames = ImmutableList.copyOf(lookupProvider.getAllLookupNames());
long lookupFootprint = 0;
@ -131,10 +164,7 @@ public class MemoryIntrospectorImpl implements MemoryIntrospector
}
}
if (logFootprint) {
log.info("Lookup footprint: lookup count[%d], total bytes[%,d].", lookupNames.size(), lookupFootprint);
}
log.info("Lookup footprint: lookup count[%d], total bytes[%,d].", lookupNames.size(), lookupFootprint);
return lookupFootprint;
}
}

View File

@ -0,0 +1,63 @@
/*
* 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.exec;
import org.apache.druid.collections.NonBlockingPool;
import org.apache.druid.collections.QueueNonBlockingPool;
import org.apache.druid.frame.processor.Bouncer;
import org.apache.druid.msq.kernel.FrameContext;
import java.nio.ByteBuffer;
import java.util.Collection;
import java.util.concurrent.ArrayBlockingQueue;
import java.util.concurrent.BlockingQueue;
/**
* Holds a processing buffer pool, and a {@link Bouncer} used to limit concurrent access to the buffer pool.
* Thread-safe. Used by {@link RunWorkOrder} by way of {@link FrameContext#processingBuffers()}.
*/
public class ProcessingBuffers
{
private final NonBlockingPool<ByteBuffer> bufferPool;
private final Bouncer bouncer;
public ProcessingBuffers(final NonBlockingPool<ByteBuffer> bufferPool, final Bouncer bouncer)
{
this.bufferPool = bufferPool;
this.bouncer = bouncer;
}
public static ProcessingBuffers fromCollection(final Collection<ByteBuffer> bufferPool)
{
final BlockingQueue<ByteBuffer> queue = new ArrayBlockingQueue<>(bufferPool.size());
queue.addAll(bufferPool);
return new ProcessingBuffers(new QueueNonBlockingPool<>(queue), new Bouncer(queue.size()));
}
public NonBlockingPool<ByteBuffer> getBufferPool()
{
return bufferPool;
}
public Bouncer getBouncer()
{
return bouncer;
}
}

View File

@ -0,0 +1,58 @@
/*
* 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.exec;
import org.apache.druid.collections.ResourceHolder;
import org.apache.druid.msq.kernel.FrameProcessorFactory;
import org.apache.druid.msq.kernel.QueryDefinition;
/**
* Provides processing buffers for {@link org.apache.druid.msq.kernel.WorkOrder}. Thread-safe, shared by all
* {@link Worker} in a particular JVM.
*/
public interface ProcessingBuffersProvider
{
/**
* Acquire buffers for a {@link Worker}.
*/
ResourceHolder<ProcessingBuffersSet> acquire(int poolSize);
/**
* Acquire buffers for a {@link Worker}, using a pool size equal to the minimum of
* {@link WorkerContext#maxConcurrentStages()} and the number of stages in the query where
* {@link FrameProcessorFactory#usesProcessingBuffers()}. (These are both caps on the number of concurrent
* stages that will need processing buffers at once.)
*/
default ResourceHolder<ProcessingBuffersSet> acquire(
final QueryDefinition queryDef,
final int maxConcurrentStages
)
{
final int poolSize = Math.min(
maxConcurrentStages,
(int) queryDef.getStageDefinitions()
.stream()
.filter(stageDef -> stageDef.getProcessorFactory().usesProcessingBuffers())
.count()
);
return acquire(poolSize);
}
}

View File

@ -0,0 +1,92 @@
/*
* 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.exec;
import org.apache.druid.collections.ResourceHolder;
import org.apache.druid.error.DruidException;
import org.apache.druid.msq.kernel.StageDefinition;
import javax.annotation.Nullable;
import java.nio.ByteBuffer;
import java.util.Collection;
import java.util.Collections;
import java.util.concurrent.ArrayBlockingQueue;
import java.util.concurrent.BlockingQueue;
import java.util.stream.Collectors;
/**
* Holds a set of {@link ProcessingBuffers} for a {@link Worker}. Acquired from {@link ProcessingBuffersProvider}.
*/
public class ProcessingBuffersSet
{
public static final ProcessingBuffersSet EMPTY = new ProcessingBuffersSet(Collections.emptyList());
private final BlockingQueue<ProcessingBuffers> pool;
public ProcessingBuffersSet(Collection<ProcessingBuffers> buffers)
{
this.pool = new ArrayBlockingQueue<>(buffers.isEmpty() ? 1 : buffers.size());
this.pool.addAll(buffers);
}
/**
* Equivalent to calling {@link ProcessingBuffers#fromCollection} on each collection in the overall collection,
* then creating an instance.
*/
public static <T extends Collection<ByteBuffer>> ProcessingBuffersSet fromCollection(final Collection<T> processingBuffers)
{
return new ProcessingBuffersSet(
processingBuffers.stream()
.map(ProcessingBuffers::fromCollection)
.collect(Collectors.toList())
);
}
@Nullable
public ResourceHolder<ProcessingBuffers> acquireForStage(final StageDefinition stageDef)
{
if (!stageDef.getProcessorFactory().usesProcessingBuffers()) {
return null;
}
final ProcessingBuffers buffers = pool.poll();
if (buffers == null) {
// Never happens, because the pool acquired from ProcessingBuffersProvider must be big enough for all
// concurrent processing buffer needs. (In other words: if this does happen, it's a bug.)
throw DruidException.defensive("Processing buffers not available");
}
return new ResourceHolder<ProcessingBuffers>()
{
@Override
public ProcessingBuffers get()
{
return buffers;
}
@Override
public void close()
{
pool.add(buffers);
}
};
}
}

View File

@ -242,7 +242,7 @@ public class RunWorkOrder
workOrder.getQueryDefinition(),
InputSlices.allReadablePartitions(workOrder.getInputs()),
inputChannelFactory,
() -> ArenaMemoryAllocator.createOnHeap(frameContext.memoryParameters().getStandardFrameSize()),
() -> ArenaMemoryAllocator.createOnHeap(frameContext.memoryParameters().getFrameSize()),
exec,
cancellationId,
counterTracker,
@ -270,18 +270,8 @@ public class RunWorkOrder
final OutputChannelFactory baseOutputChannelFactory;
if (workOrder.getStageDefinition().doesShuffle()) {
// Writing to a consumer in the same JVM (which will be set up later on in this method). Use the large frame
// size if we're writing to a SuperSorter, since we'll generate fewer temp files if we use larger frames.
// Otherwise, use the standard frame size.
final int frameSize;
if (workOrder.getStageDefinition().getShuffleSpec().kind().isSort()) {
frameSize = frameContext.memoryParameters().getLargeFrameSize();
} else {
frameSize = frameContext.memoryParameters().getStandardFrameSize();
}
baseOutputChannelFactory = new BlockingQueueOutputChannelFactory(frameSize);
// Writing to a consumer in the same JVM (which will be set up later on in this method).
baseOutputChannelFactory = new BlockingQueueOutputChannelFactory(frameContext.memoryParameters().getFrameSize());
} else {
// Writing stage output.
baseOutputChannelFactory = makeStageOutputChannelFactory();
@ -353,7 +343,7 @@ public class RunWorkOrder
final ListenableFuture<ManagerReturnType> workResultFuture = exec.runAllFully(
counterTracker.trackCpu(processorManager, CpuCounters.LABEL_MAIN),
maxOutstandingProcessors,
frameContext.processorBouncer(),
processorFactory.usesProcessingBuffers() ? frameContext.processingBuffers().getBouncer() : Bouncer.unlimited(),
cancellationId
);
@ -394,13 +384,13 @@ public class RunWorkOrder
if (shuffleSpec.partitionCount() == 1) {
// Single partition; no need to write temporary files.
hashOutputChannelFactory =
new BlockingQueueOutputChannelFactory(frameContext.memoryParameters().getStandardFrameSize());
new BlockingQueueOutputChannelFactory(frameContext.memoryParameters().getFrameSize());
} else {
// Multi-partition; write temporary files and then sort each one file-by-file.
hashOutputChannelFactory =
new FileOutputChannelFactory(
frameContext.tempDir("hash-parts"),
frameContext.memoryParameters().getStandardFrameSize(),
frameContext.memoryParameters().getFrameSize(),
null
);
}
@ -490,7 +480,7 @@ public class RunWorkOrder
final DurableStorageOutputChannelFactory durableStorageOutputChannelFactory =
makeDurableStorageOutputChannelFactory(
frameContext.tempDir("durable"),
frameContext.memoryParameters().getStandardFrameSize(),
frameContext.memoryParameters().getFrameSize(),
workOrder.getOutputChannelMode() == OutputChannelMode.DURABLE_STORAGE_QUERY_RESULTS
);
@ -510,7 +500,7 @@ public class RunWorkOrder
{
// Use the standard frame size, since we assume this size when computing how much is needed to merge output
// files from different workers.
final int frameSize = frameContext.memoryParameters().getStandardFrameSize();
final int frameSize = frameContext.memoryParameters().getFrameSize();
final OutputChannelMode outputChannelMode = workOrder.getOutputChannelMode();
switch (outputChannelMode) {
@ -542,7 +532,7 @@ public class RunWorkOrder
private OutputChannelFactory makeSuperSorterIntermediateOutputChannelFactory(final File tmpDir)
{
final int frameSize = frameContext.memoryParameters().getLargeFrameSize();
final int frameSize = frameContext.memoryParameters().getFrameSize();
final File fileChannelDirectory =
new File(tmpDir, StringUtils.format("intermediate_output_stage_%06d", workOrder.getStageNumber()));
final FileOutputChannelFactory fileOutputChannelFactory =
@ -736,8 +726,8 @@ public class RunWorkOrder
},
outputChannelFactory,
makeSuperSorterIntermediateOutputChannelFactory(sorterTmpDir),
memoryParameters.getSuperSorterMaxActiveProcessors(),
memoryParameters.getSuperSorterMaxChannelsPerProcessor(),
memoryParameters.getSuperSorterConcurrentProcessors(),
memoryParameters.getSuperSorterMaxChannelsPerMerger(),
stageDefinition.getShuffleSpec().limitHint(),
cancellationId,
counterTracker.sortProgress(),
@ -774,7 +764,7 @@ public class RunWorkOrder
workOrder.getStageDefinition().getFrameReader(),
workOrder.getStageDefinition().getClusterBy().getColumns().size(),
FrameWriters.makeRowBasedFrameWriterFactory(
new ArenaMemoryAllocatorFactory(frameContext.memoryParameters().getStandardFrameSize()),
new ArenaMemoryAllocatorFactory(frameContext.memoryParameters().getFrameSize()),
workOrder.getStageDefinition().getSignature(),
workOrder.getStageDefinition().getSortKey(),
removeNullBytes

View File

@ -25,7 +25,6 @@ import org.apache.druid.java.util.common.io.Closer;
import org.apache.druid.msq.indexing.MSQWorkerTask;
import org.apache.druid.msq.kernel.FrameContext;
import org.apache.druid.msq.kernel.FrameProcessorFactory;
import org.apache.druid.msq.kernel.QueryDefinition;
import org.apache.druid.msq.kernel.WorkOrder;
import org.apache.druid.server.DruidNode;
@ -78,14 +77,15 @@ public interface WorkerContext
WorkerClient makeWorkerClient();
/**
* Directory for temporary outputs.
* Directory for temporary outputs, used as a base for {@link FrameContext#tempDir()}. This directory is not
* necessarily fully owned by the worker.
*/
File tempDir();
/**
* Create a context with useful objects required by {@link FrameProcessorFactory#makeProcessors}.
*/
FrameContext frameContext(QueryDefinition queryDef, int stageNumber, OutputChannelMode outputChannelMode);
FrameContext frameContext(WorkOrder workOrder);
/**
* Number of available processing threads.

View File

@ -376,13 +376,7 @@ public class WorkerImpl implements Worker
? StringUtils.format(", payload[%s]", context.jsonMapper().writeValueAsString(workOrder)) : "")
);
final FrameContext frameContext = kernelHolder.processorCloser.register(
context.frameContext(
workOrder.getQueryDefinition(),
stageDefinition.getStageNumber(),
workOrder.getOutputChannelMode()
)
);
final FrameContext frameContext = kernelHolder.processorCloser.register(context.frameContext(workOrder));
kernelHolder.processorCloser.register(() -> {
try {
workerExec.cancel(cancellationId);

View File

@ -19,92 +19,66 @@
package org.apache.druid.msq.exec;
import com.google.common.base.Preconditions;
import com.google.common.collect.Iterables;
import com.google.common.primitives.Ints;
import com.google.inject.Injector;
import it.unimi.dsi.fastutil.ints.IntOpenHashSet;
import it.unimi.dsi.fastutil.ints.IntSet;
import org.apache.druid.frame.processor.Bouncer;
import org.apache.druid.indexing.worker.config.WorkerConfig;
import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.frame.processor.FrameProcessor;
import org.apache.druid.frame.processor.SuperSorter;
import org.apache.druid.msq.indexing.error.MSQException;
import org.apache.druid.msq.indexing.error.NotEnoughMemoryFault;
import org.apache.druid.msq.indexing.error.TooManyWorkersFault;
import org.apache.druid.msq.input.InputSpecs;
import org.apache.druid.msq.kernel.QueryDefinition;
import org.apache.druid.msq.indexing.processor.KeyStatisticsCollectionProcessor;
import org.apache.druid.msq.indexing.processor.SegmentGeneratorFrameProcessorFactory;
import org.apache.druid.msq.input.InputSlice;
import org.apache.druid.msq.input.InputSlices;
import org.apache.druid.msq.input.stage.ReadablePartition;
import org.apache.druid.msq.input.stage.StageInputSlice;
import org.apache.druid.msq.kernel.GlobalSortMaxCountShuffleSpec;
import org.apache.druid.msq.kernel.ShuffleSpec;
import org.apache.druid.msq.kernel.StageDefinition;
import org.apache.druid.msq.kernel.WorkOrder;
import org.apache.druid.msq.querykit.BroadcastJoinSegmentMapFnProcessor;
import org.apache.druid.msq.statistics.ClusterByStatisticsCollectorImpl;
import org.apache.druid.query.lookup.LookupExtractor;
import org.apache.druid.query.lookup.LookupExtractorFactoryContainer;
import org.apache.druid.query.lookup.LookupExtractorFactoryContainerProvider;
import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager;
import org.apache.druid.segment.realtime.appenderator.UnifiedIndexerAppenderatorsManager;
import org.apache.druid.segment.incremental.IncrementalIndex;
import javax.annotation.Nullable;
import java.util.List;
import java.util.Objects;
/**
* Class for determining how much JVM heap to allocate to various purposes.
* Class for determining how much JVM heap to allocate to various purposes for executing a {@link WorkOrder}.
*
* First, we take a chunk out of the total JVM heap that is dedicated for MSQ; see {@link #computeUsableMemoryInJvm}.
* First, we split each worker's memory allotment, given by {@link MemoryIntrospector#memoryPerTask()}, into
* equally-sized "bundles" for each {@link WorkOrder} that may be running simultaneously within the {@link Worker}
* for that {@link WorkOrder}.
*
* Then, we carve out some space for each worker that may be running in our JVM; see {@link #memoryPerWorker}.
* Within each bundle, we carve out memory required for buffering broadcast data
* (see {@link #computeBroadcastBufferMemory}) and for concurrently-running processors
* (see {@link #computeProcessorMemory}).
*
* Then, we split the rest into "bundles" of equal size; see {@link #memoryPerBundle}. The number of bundles is based
* entirely on server configuration; this makes the calculation robust to different queries running simultaneously in
* the same JVM.
*
* Within each bundle, we split up memory in two different ways: one assuming it'll be used for a
* {@link org.apache.druid.frame.processor.SuperSorter}, and one assuming it'll be used for a regular
* processor. Callers can then use whichever set of allocations makes sense. (We assume no single bundle
* will be used for both purposes.)
* The remainder is called "bundle free memory", a pool of memory that can be used for {@link SuperSorter} or
* {@link SegmentGeneratorFrameProcessorFactory}. The amounts overlap, because the same {@link WorkOrder} never
* does both.
*/
public class WorkerMemoryParameters
{
private static final Logger log = new Logger(WorkerMemoryParameters.class);
/**
* Percent of memory that we allocate to bundles. It is less than 100% because we need to leave some space
* left over for miscellaneous other stuff, and to ensure that GC pressure does not get too high.
* Default size for frames.
*/
static final double USABLE_MEMORY_FRACTION = 0.75;
public static final int DEFAULT_FRAME_SIZE = 1_000_000;
/**
* Percent of each bundle's memory that we allocate to appenderators. It is less than 100% because appenderators
* Amount of extra memory available for each processing thread, beyond what is needed for input and output
* channels. This memory is used for miscellaneous purposes within the various {@link FrameProcessor}.
*/
private static final long EXTRA_MEMORY_PER_PROCESSOR = 25_000_000;
/**
* Percent of each bundle's free memory that we allocate to appenderators. It is less than 100% because appenderators
* unfortunately have a variety of unaccounted-for memory usage.
*/
static final double APPENDERATOR_MEMORY_FRACTION = 0.67;
/**
* Size for "standard frames", which are used for most purposes, except inputs to super-sorters.
*
* In particular, frames that travel between workers are always the minimum size. This is helpful because it makes
* it easier to compute the amount of memory needed to merge input streams.
*/
private static final int STANDARD_FRAME_SIZE = 1_000_000;
/**
* Size for "large frames", which are used for inputs and inner channels in to super-sorters.
*
* This is helpful because it minimizes the number of temporary files needed during super-sorting.
*/
private static final int LARGE_FRAME_SIZE = 8_000_000;
/**
* Minimum amount of bundle memory available for processing (i.e., total bundle size minus the amount
* needed for input channels). This memory is guaranteed to be available for things like segment generation
* and broadcast data.
*/
public static final long PROCESSING_MINIMUM_BYTES = 25_000_000;
/**
* Maximum amount of parallelism for the super-sorter. Higher amounts of concurrency tend to be wasteful.
*/
private static final int MAX_SUPER_SORTER_PROCESSORS = 4;
/**
* Each super-sorter must have at least 1 processor with 2 input frames and 1 output frame. That's 3 total.
*/
private static final int MIN_SUPER_SORTER_FRAMES = 3;
private static final double APPENDERATOR_BUNDLE_FREE_MEMORY_FRACTION = 0.67;
/**
* (Very) rough estimate of the on-heap overhead of reading a column.
@ -112,256 +86,214 @@ public class WorkerMemoryParameters
private static final int APPENDERATOR_MERGE_ROUGH_MEMORY_PER_COLUMN = 3_000;
/**
* Maximum percent of *total* available memory (not each bundle), i.e. {@link #USABLE_MEMORY_FRACTION}, that we'll
* ever use for maxRetainedBytes of {@link ClusterByStatisticsCollectorImpl} across all workers.
* Maximum percent of each bundle's free memory that will be used for maxRetainedBytes of
* {@link ClusterByStatisticsCollectorImpl}.
*/
private static final double PARTITION_STATS_MEMORY_MAX_FRACTION = 0.1;
private static final double PARTITION_STATS_MAX_BUNDLE_FREE_MEMORY_FRACTION = 0.1;
/**
* Maximum number of bytes we'll ever use for maxRetainedBytes of {@link ClusterByStatisticsCollectorImpl} for
* a single worker. Acts as a limit on the value computed based on {@link #PARTITION_STATS_MEMORY_MAX_FRACTION}.
* Maximum number of bytes from each bundle's free memory that we'll ever use for maxRetainedBytes of
* {@link ClusterByStatisticsCollectorImpl}. Limits the value computed based on
* {@link #PARTITION_STATS_MAX_BUNDLE_FREE_MEMORY_FRACTION}.
*/
private static final long PARTITION_STATS_MEMORY_MAX_BYTES = 300_000_000;
private static final long PARTITION_STATS_MAX_MEMORY_PER_BUNDLE = 300_000_000;
/**
* Threshold in bytes below which we assume that the worker is "small". While calculating the memory requirements for
* a small worker, we try to be as conservatives with the estimates and the extra temporary space required by the
* frames, since that can add up quickly and cause OOM.
* Minimum number of bytes from each bundle's free memory that we'll use for maxRetainedBytes of
* {@link ClusterByStatisticsCollectorImpl}.
*/
private static final long SMALL_WORKER_CAPACITY_THRESHOLD_BYTES = 256_000_000;
private static final long PARTITION_STATS_MIN_MEMORY_PER_BUNDLE = 10_000_000;
/**
* Fraction of free memory per bundle that can be used by {@link BroadcastJoinSegmentMapFnProcessor} to store broadcast
* data on-heap. This is used to limit the total size of input frames, which we expect to expand on-heap. Expansion
* can potentially be somewhat over 2x: for example, strings are UTF-8 in frames, but are UTF-16 on-heap, which is
* a 2x expansion, and object and index overhead must be considered on top of that. So we use a value somewhat
* lower than 0.5.
* Fraction of each bundle's total memory that can be used to buffer broadcast inputs. This is used by
* {@link BroadcastJoinSegmentMapFnProcessor} to limit how much joinable data is stored on-heap. This is carved
* directly out of the total bundle memory, which makes its size more predictable and stable: it only depends on
* the total JVM memory, the number of tasks per JVM, and the value of maxConcurrentStages for the query. This
* stability is important, because if the broadcast buffer fills up, the query fails. So any time its size changes,
* we risk queries failing that would formerly have succeeded.
*/
static final double BROADCAST_JOIN_MEMORY_FRACTION = 0.3;
private static final double BROADCAST_BUFFER_TOTAL_MEMORY_FRACTION = 0.2;
/**
* Fraction of free memory per bundle that can be used by
* Multiplier to apply to {@link #BROADCAST_BUFFER_TOTAL_MEMORY_FRACTION} when determining how much free bundle
* memory is left over. This fudge factor exists because {@link BroadcastJoinSegmentMapFnProcessor} applies data
* size limits based on frame size, which we expect to expand somewhat in memory due to indexing structures in
* {@link org.apache.druid.segment.join.table.FrameBasedIndexedTable}.
*/
private static final double BROADCAST_BUFFER_OVERHEAD_RATIO = 1.5;
/**
* Amount of memory that can be used by
* {@link org.apache.druid.msq.querykit.common.SortMergeJoinFrameProcessor} to buffer frames in its trackers.
*/
static final double SORT_MERGE_JOIN_MEMORY_FRACTION = 0.9;
private static final long SORT_MERGE_JOIN_MEMORY_PER_PROCESSOR = (long) (EXTRA_MEMORY_PER_PROCESSOR * 0.9);
/**
* In case {@link NotEnoughMemoryFault} is thrown, a fixed estimation overhead is added when estimating total memory required for the process.
*/
private static final long BUFFER_BYTES_FOR_ESTIMATION = 1000;
private final long processorBundleMemory;
private final int superSorterMaxActiveProcessors;
private final int superSorterMaxChannelsPerProcessor;
private final long bundleFreeMemory;
private final int frameSize;
private final int superSorterConcurrentProcessors;
private final int superSorterMaxChannelsPerMerger;
private final int partitionStatisticsMaxRetainedBytes;
private final long broadcastBufferMemory;
WorkerMemoryParameters(
final long processorBundleMemory,
final int superSorterMaxActiveProcessors,
final int superSorterMaxChannelsPerProcessor,
final int partitionStatisticsMaxRetainedBytes
public WorkerMemoryParameters(
final long bundleFreeMemory,
final int frameSize,
final int superSorterConcurrentProcessors,
final int superSorterMaxChannelsPerMerger,
final int partitionStatisticsMaxRetainedBytes,
final long broadcastBufferMemory
)
{
this.processorBundleMemory = processorBundleMemory;
this.superSorterMaxActiveProcessors = superSorterMaxActiveProcessors;
this.superSorterMaxChannelsPerProcessor = superSorterMaxChannelsPerProcessor;
this.bundleFreeMemory = bundleFreeMemory;
this.frameSize = frameSize;
this.superSorterConcurrentProcessors = superSorterConcurrentProcessors;
this.superSorterMaxChannelsPerMerger = superSorterMaxChannelsPerMerger;
this.partitionStatisticsMaxRetainedBytes = partitionStatisticsMaxRetainedBytes;
this.broadcastBufferMemory = broadcastBufferMemory;
}
/**
* Create a production instance for {@link org.apache.druid.msq.indexing.MSQWorkerTask}.
* Create a production instance for a given {@link WorkOrder}.
*/
public static WorkerMemoryParameters createProductionInstanceForWorker(
final Injector injector,
final QueryDefinition queryDef,
final int stageNumber,
public static WorkerMemoryParameters createProductionInstance(
final WorkOrder workOrder,
final MemoryIntrospector memoryIntrospector,
final int maxConcurrentStages
)
{
final StageDefinition stageDef = queryDef.getStageDefinition(stageNumber);
final IntSet inputStageNumbers = InputSpecs.getStageNumbers(stageDef.getInputSpecs());
final int numInputWorkers =
inputStageNumbers.intStream()
.map(inputStageNumber -> queryDef.getStageDefinition(inputStageNumber).getMaxWorkerCount())
.sum();
long totalLookupFootprint = computeTotalLookupFootprint(injector);
final int numHashOutputPartitions;
if (stageDef.doesShuffle() && stageDef.getShuffleSpec().kind().isHash()) {
numHashOutputPartitions = stageDef.getShuffleSpec().partitionCount();
} else {
numHashOutputPartitions = 0;
}
final StageDefinition stageDef = workOrder.getStageDefinition();
return createInstance(
Runtime.getRuntime().maxMemory(),
computeNumWorkersInJvm(injector),
computeNumProcessorsInJvm(injector),
memoryIntrospector,
DEFAULT_FRAME_SIZE,
workOrder.getInputs(),
stageDef.getBroadcastInputNumbers(),
stageDef.doesShuffle() ? stageDef.getShuffleSpec() : null,
maxConcurrentStages,
numInputWorkers,
numHashOutputPartitions,
totalLookupFootprint
computeFramesPerOutputChannel(workOrder.getOutputChannelMode())
);
}
/**
* Returns an object specifying memory-usage parameters.
* Returns an object specifying memory-usage parameters for a {@link WorkOrder} running inside a {@link Worker}.
*
* Throws a {@link MSQException} with an appropriate fault if the provided combination of parameters cannot
* yield a workable memory situation.
*
* @param maxMemoryInJvm memory available in the entire JVM. This will be divided amongst processors.
* @param numWorkersInJvm number of workers that can run concurrently in this JVM. Generally equal to
* the task capacity.
* @param numProcessingThreadsInJvm size of the processing thread pool in the JVM.
* @param maxConcurrentStages maximum number of concurrent stages per worker.
* @param numInputWorkers total number of workers across all input stages.
* @param numHashOutputPartitions total number of output partitions, if using hash partitioning; zero if not using
* hash partitioning.
* @param totalLookupFootprint estimated size of the lookups loaded by the process.
* @param memoryIntrospector memory introspector
* @param frameSize frame size
* @param inputSlices from {@link WorkOrder#getInputs()}
* @param broadcastInputNumbers from {@link StageDefinition#getBroadcastInputNumbers()}
* @param shuffleSpec from {@link StageDefinition#getShuffleSpec()}
* @param maxConcurrentStages figure from {@link WorkerContext#maxConcurrentStages()}
* @param numFramesPerOutputChannel figure from {@link #computeFramesPerOutputChannel(OutputChannelMode)}
*
* @throws MSQException with {@link TooManyWorkersFault} or {@link NotEnoughMemoryFault} if not enough memory
* is available to generate a usable instance
*/
public static WorkerMemoryParameters createInstance(
final long maxMemoryInJvm,
final int numWorkersInJvm,
final int numProcessingThreadsInJvm,
final MemoryIntrospector memoryIntrospector,
final int frameSize,
final List<InputSlice> inputSlices,
final IntSet broadcastInputNumbers,
@Nullable final ShuffleSpec shuffleSpec,
final int maxConcurrentStages,
final int numInputWorkers,
final int numHashOutputPartitions,
final long totalLookupFootprint
final int numFramesPerOutputChannel
)
{
Preconditions.checkArgument(maxMemoryInJvm > 0, "Max memory passed: [%s] should be > 0", maxMemoryInJvm);
Preconditions.checkArgument(numWorkersInJvm > 0, "Number of workers: [%s] in jvm should be > 0", numWorkersInJvm);
Preconditions.checkArgument(
numProcessingThreadsInJvm > 0,
"Number of processing threads [%s] should be > 0",
numProcessingThreadsInJvm
final long bundleMemory = computeBundleMemory(memoryIntrospector.memoryPerTask(), maxConcurrentStages);
final long processorMemory = computeProcessorMemory(
computeMaxSimultaneousInputChannelsPerProcessor(inputSlices, broadcastInputNumbers),
frameSize
);
Preconditions.checkArgument(numInputWorkers >= 0, "Number of input workers: [%s] should be >=0", numInputWorkers);
Preconditions.checkArgument(
totalLookupFootprint >= 0,
"Lookup memory footprint: [%s] should be >= 0",
totalLookupFootprint
);
final long usableMemoryInJvm = computeUsableMemoryInJvm(maxMemoryInJvm, totalLookupFootprint);
final long workerMemory = memoryPerWorker(usableMemoryInJvm, numWorkersInJvm);
final long bundleMemory =
memoryPerBundle(usableMemoryInJvm, numWorkersInJvm, numProcessingThreadsInJvm) / maxConcurrentStages;
final long bundleMemoryForInputChannels = memoryNeededForInputChannels(numInputWorkers);
final long bundleMemoryForHashPartitioning = memoryNeededForHashPartitioning(numHashOutputPartitions);
final long bundleMemoryForProcessing =
bundleMemory - bundleMemoryForInputChannels - bundleMemoryForHashPartitioning;
final boolean hasBroadcastInputs = !broadcastInputNumbers.isEmpty();
final long broadcastBufferMemory =
hasBroadcastInputs ? computeBroadcastBufferMemoryIncludingOverhead(bundleMemory) : 0;
final int numProcessingThreads = memoryIntrospector.numProcessingThreads();
final int maxSimultaneousWorkProcessors = Math.min(numProcessingThreads, computeNumInputPartitions(inputSlices));
final long bundleFreeMemory =
bundleMemory - maxSimultaneousWorkProcessors * processorMemory - broadcastBufferMemory;
if (bundleMemoryForProcessing < PROCESSING_MINIMUM_BYTES) {
final int maxWorkers = computeMaxWorkers(
usableMemoryInJvm,
numWorkersInJvm,
numProcessingThreadsInJvm,
maxConcurrentStages,
numHashOutputPartitions
);
if (maxWorkers > 0) {
throw new MSQException(new TooManyWorkersFault(numInputWorkers, Math.min(Limits.MAX_WORKERS, maxWorkers)));
} else {
// Not enough memory for even one worker. More of a NotEnoughMemory situation than a TooManyWorkers situation.
throw new MSQException(
new NotEnoughMemoryFault(
calculateSuggestedMinMemoryFromUsableMemory(
estimateUsableMemory(
numWorkersInJvm,
numProcessingThreadsInJvm,
PROCESSING_MINIMUM_BYTES + BUFFER_BYTES_FOR_ESTIMATION + bundleMemoryForInputChannels,
maxConcurrentStages
), totalLookupFootprint),
maxMemoryInJvm,
usableMemoryInJvm,
numWorkersInJvm,
numProcessingThreadsInJvm,
maxConcurrentStages
)
);
}
}
// Compute memory breakdown for super-sorting bundles.
final int maxNumFramesForSuperSorter = Ints.checkedCast(bundleMemory / WorkerMemoryParameters.LARGE_FRAME_SIZE);
if (maxNumFramesForSuperSorter < MIN_SUPER_SORTER_FRAMES) {
final long minimumBundleFreeMemory = computeMinimumBundleFreeMemory(frameSize, numFramesPerOutputChannel);
if (bundleFreeMemory < minimumBundleFreeMemory) {
final long requiredTaskMemory = bundleMemory - bundleFreeMemory + minimumBundleFreeMemory;
throw new MSQException(
new NotEnoughMemoryFault(
calculateSuggestedMinMemoryFromUsableMemory(
estimateUsableMemory(
numWorkersInJvm,
(MIN_SUPER_SORTER_FRAMES + BUFFER_BYTES_FOR_ESTIMATION) * LARGE_FRAME_SIZE,
maxConcurrentStages
),
totalLookupFootprint
),
maxMemoryInJvm,
usableMemoryInJvm,
numWorkersInJvm,
numProcessingThreadsInJvm,
memoryIntrospector.computeJvmMemoryRequiredForTaskMemory(requiredTaskMemory),
memoryIntrospector.totalMemoryInJvm(),
memoryIntrospector.memoryPerTask(),
memoryIntrospector.numTasksInJvm(),
memoryIntrospector.numProcessingThreads(),
computeNumInputWorkers(inputSlices),
maxConcurrentStages
)
);
}
final int superSorterMaxActiveProcessors = Math.min(
numProcessingThreadsInJvm,
Math.min(
maxNumFramesForSuperSorter / MIN_SUPER_SORTER_FRAMES,
MAX_SUPER_SORTER_PROCESSORS
)
);
// Compute memory breakdown for super-sorting bundles.
final int partitionStatsMemory =
StageDefinition.mustGatherResultKeyStatistics(shuffleSpec) ? computePartitionStatsMemory(bundleFreeMemory) : 0;
final long superSorterMemory = bundleFreeMemory - partitionStatsMemory;
final int maxOutputPartitions = computeMaxOutputPartitions(shuffleSpec);
final int isSmallWorker = usableMemoryInJvm < SMALL_WORKER_CAPACITY_THRESHOLD_BYTES ? 1 : 0;
// Apportion max frames to all processors equally, then subtract one to account for an output frame and one to account
// for the durable storage's output frame in the supersorter. The extra frame is required in case of durable storage
// since composing output channel factories keep a frame open while writing to them.
// We only account for this extra frame in the workers where the heap size is relatively small to be more
// conservative with the memory estimations. In workers with heap size larger than the frame size, we can get away
// without accounting for this extra frame, and instead better parallelize the supersorter's operations.
final int superSorterMaxChannelsPerProcessor = maxNumFramesForSuperSorter / superSorterMaxActiveProcessors
- 1
- isSmallWorker;
if (superSorterMaxActiveProcessors <= 0) {
int superSorterConcurrentProcessors;
int superSorterMaxChannelsPerMerger = -1;
if (maxOutputPartitions == 0) {
superSorterConcurrentProcessors = numProcessingThreads;
} else {
superSorterConcurrentProcessors = Math.min(maxOutputPartitions, numProcessingThreads);
}
for (; superSorterConcurrentProcessors > 0; superSorterConcurrentProcessors--) {
final long memoryPerProcessor = superSorterMemory / superSorterConcurrentProcessors;
// Each processor has at least 2 frames for inputs, plus numFramesPerOutputChannel for outputs.
// Compute whether we can support this level of parallelism, given these constraints.
final int minMemoryForInputsPerProcessor = 2 * frameSize;
final int memoryForOutputsPerProcessor = numFramesPerOutputChannel * frameSize;
if (memoryPerProcessor >= minMemoryForInputsPerProcessor + memoryForOutputsPerProcessor) {
final long memoryForInputsPerProcessor = memoryPerProcessor - memoryForOutputsPerProcessor;
superSorterMaxChannelsPerMerger = Ints.checkedCast(memoryForInputsPerProcessor / frameSize);
break;
}
}
if (superSorterConcurrentProcessors == 0) {
// Couldn't support any level of concurrency. Not expected, since we should have accounted for at least a
// minimally-sized SuperSorter by way of the calculation in "computeMinimumBundleFreeMemory". Return a
// NotEnoughMemoryFault with no suggestedServerMemory, since at this point, we aren't sure what will work.
throw new MSQException(
new NotEnoughMemoryFault(
calculateSuggestedMinMemoryFromUsableMemory(
estimateUsableMemory(
numWorkersInJvm,
numProcessingThreadsInJvm,
PROCESSING_MINIMUM_BYTES + BUFFER_BYTES_FOR_ESTIMATION + bundleMemoryForInputChannels,
maxConcurrentStages
), totalLookupFootprint),
maxMemoryInJvm,
usableMemoryInJvm,
numWorkersInJvm,
numProcessingThreadsInJvm,
0,
memoryIntrospector.totalMemoryInJvm(),
memoryIntrospector.memoryPerTask(),
memoryIntrospector.numTasksInJvm(),
memoryIntrospector.numProcessingThreads(),
computeNumInputWorkers(inputSlices),
maxConcurrentStages
)
);
}
return new WorkerMemoryParameters(
bundleMemoryForProcessing,
superSorterMaxActiveProcessors,
superSorterMaxChannelsPerProcessor,
// 100% of worker memory is devoted to partition statistics
Ints.checkedCast(workerMemory / maxConcurrentStages)
bundleFreeMemory,
frameSize,
superSorterConcurrentProcessors,
superSorterMaxChannelsPerMerger,
Math.min(Integer.MAX_VALUE, partitionStatsMemory / numProcessingThreads),
hasBroadcastInputs ? computeBroadcastBufferMemory(bundleMemory) : 0
);
}
public int getSuperSorterMaxActiveProcessors()
public int getSuperSorterConcurrentProcessors()
{
return superSorterMaxActiveProcessors;
return superSorterConcurrentProcessors;
}
public int getSuperSorterMaxChannelsPerProcessor()
public int getSuperSorterMaxChannelsPerMerger()
{
return superSorterMaxChannelsPerProcessor;
return superSorterMaxChannelsPerMerger;
}
public long getAppenderatorMaxBytesInMemory()
@ -376,24 +308,27 @@ public class WorkerMemoryParameters
return Ints.checkedCast(Math.max(2, getAppenderatorMemory() / 2 / APPENDERATOR_MERGE_ROUGH_MEMORY_PER_COLUMN));
}
public int getStandardFrameSize()
public int getFrameSize()
{
return STANDARD_FRAME_SIZE;
return frameSize;
}
public int getLargeFrameSize()
/**
* Memory available for buffering broadcast data. Used to restrict the amount of memory used by
* {@link BroadcastJoinSegmentMapFnProcessor}.
*/
public long getBroadcastBufferMemory()
{
return LARGE_FRAME_SIZE;
}
public long getBroadcastJoinMemory()
{
return (long) (processorBundleMemory * BROADCAST_JOIN_MEMORY_FRACTION);
return broadcastBufferMemory;
}
/**
* Fraction of each processor's memory that can be used by
* {@link org.apache.druid.msq.querykit.common.SortMergeJoinFrameProcessor} to buffer frames in its trackers.
*/
public long getSortMergeJoinMemory()
{
return (long) (processorBundleMemory * SORT_MERGE_JOIN_MEMORY_FRACTION);
return SORT_MERGE_JOIN_MEMORY_PER_PROCESSOR;
}
public int getPartitionStatisticsMaxRetainedBytes()
@ -406,7 +341,7 @@ public class WorkerMemoryParameters
*/
private long getAppenderatorMemory()
{
return (long) (processorBundleMemory * APPENDERATOR_MEMORY_FRACTION);
return (long) (bundleFreeMemory * APPENDERATOR_BUNDLE_FREE_MEMORY_FRACTION);
}
@Override
@ -419,20 +354,24 @@ public class WorkerMemoryParameters
return false;
}
WorkerMemoryParameters that = (WorkerMemoryParameters) o;
return processorBundleMemory == that.processorBundleMemory
&& superSorterMaxActiveProcessors == that.superSorterMaxActiveProcessors
&& superSorterMaxChannelsPerProcessor == that.superSorterMaxChannelsPerProcessor
&& partitionStatisticsMaxRetainedBytes == that.partitionStatisticsMaxRetainedBytes;
return bundleFreeMemory == that.bundleFreeMemory
&& frameSize == that.frameSize
&& superSorterConcurrentProcessors == that.superSorterConcurrentProcessors
&& superSorterMaxChannelsPerMerger == that.superSorterMaxChannelsPerMerger
&& partitionStatisticsMaxRetainedBytes == that.partitionStatisticsMaxRetainedBytes
&& broadcastBufferMemory == that.broadcastBufferMemory;
}
@Override
public int hashCode()
{
return Objects.hash(
processorBundleMemory,
superSorterMaxActiveProcessors,
superSorterMaxChannelsPerProcessor,
partitionStatisticsMaxRetainedBytes
bundleFreeMemory,
frameSize,
superSorterConcurrentProcessors,
superSorterMaxChannelsPerMerger,
partitionStatisticsMaxRetainedBytes,
broadcastBufferMemory
);
}
@ -440,206 +379,205 @@ public class WorkerMemoryParameters
public String toString()
{
return "WorkerMemoryParameters{" +
"processorBundleMemory=" + processorBundleMemory +
", superSorterMaxActiveProcessors=" + superSorterMaxActiveProcessors +
", superSorterMaxChannelsPerProcessor=" + superSorterMaxChannelsPerProcessor +
"bundleFreeMemory=" + bundleFreeMemory +
", frameSize=" + frameSize +
", superSorterConcurrentProcessors=" + superSorterConcurrentProcessors +
", superSorterMaxChannelsPerMerger=" + superSorterMaxChannelsPerMerger +
", partitionStatisticsMaxRetainedBytes=" + partitionStatisticsMaxRetainedBytes +
", broadcastBufferMemory=" + broadcastBufferMemory +
'}';
}
/**
* Computes the highest value of numInputWorkers, for the given parameters, that can be passed to
* {@link #createInstance} without resulting in a {@link TooManyWorkersFault}.
*
* Returns 0 if no number of workers would be OK.
* Compute the memory allocated to each {@link WorkOrder} within a {@link Worker}.
*/
static int computeMaxWorkers(
final long usableMemoryInJvm,
final int numWorkersInJvm,
final int numProcessingThreadsInJvm,
final int maxConcurrentStages,
final int numHashOutputPartitions
static long computeBundleMemory(final long memoryPerWorker, final int maxConcurrentStages)
{
return memoryPerWorker / maxConcurrentStages;
}
/**
* Compute the memory allocated to {@link KeyStatisticsCollectionProcessor} within each bundle.
*/
static int computePartitionStatsMemory(final long bundleFreeMemory)
{
return Ints.checkedCast(
Math.max(
(long) Math.min(
bundleFreeMemory * PARTITION_STATS_MAX_BUNDLE_FREE_MEMORY_FRACTION,
PARTITION_STATS_MAX_MEMORY_PER_BUNDLE
),
PARTITION_STATS_MIN_MEMORY_PER_BUNDLE
)
);
}
/**
* Compute the memory limit passed to {@link BroadcastJoinSegmentMapFnProcessor} within each worker bundle. This
* is somewhat lower than {@link #computeBroadcastBufferMemoryIncludingOverhead}, because we expect some overhead on
* top of this limit due to indexing structures. This overhead isn't accounted for by the processor
* {@link BroadcastJoinSegmentMapFnProcessor} itself.
*/
static long computeBroadcastBufferMemory(final long bundleMemory)
{
return (long) (bundleMemory * BROADCAST_BUFFER_TOTAL_MEMORY_FRACTION);
}
/**
* Memory allocated to {@link BroadcastJoinSegmentMapFnProcessor} within each worker bundle, including
* expected overhead.
*/
static long computeBroadcastBufferMemoryIncludingOverhead(final long bundleMemory)
{
return (long) (computeBroadcastBufferMemory(bundleMemory) * BROADCAST_BUFFER_OVERHEAD_RATIO);
}
/**
* Memory allocated to each processor within a bundle, including fixed overheads and buffered input and output frames.
*
* @param maxSimultaneousInputChannelsPerProcessor figure from {@link #computeMaxSimultaneousInputChannelsPerProcessor}
* @param frameSize frame size
*/
static long computeProcessorMemory(final int maxSimultaneousInputChannelsPerProcessor, final int frameSize)
{
return EXTRA_MEMORY_PER_PROCESSOR
+ computeProcessorMemoryForInputChannels(maxSimultaneousInputChannelsPerProcessor, frameSize)
+ frameSize /* output frame */;
}
/**
* Memory allocated to each processor for reading its inputs.
*
* @param maxSimultaneousInputChannelsPerProcessor figure from {@link #computeMaxSimultaneousInputChannelsPerProcessor}
* @param frameSize frame size
*/
static long computeProcessorMemoryForInputChannels(
final int maxSimultaneousInputChannelsPerProcessor,
final int frameSize
)
{
final long bundleMemory = memoryPerBundle(usableMemoryInJvm, numWorkersInJvm, numProcessingThreadsInJvm);
// Compute number of workers that gives us PROCESSING_MINIMUM_BYTES of memory per bundle per concurrent stage, while
// accounting for memoryNeededForInputChannels + memoryNeededForHashPartitioning.
final int isHashing = numHashOutputPartitions > 0 ? 1 : 0;
final long bundleMemoryPerStage = bundleMemory / maxConcurrentStages;
final long maxWorkers =
(bundleMemoryPerStage - PROCESSING_MINIMUM_BYTES) / ((long) STANDARD_FRAME_SIZE * (1 + isHashing)) - 1;
return Math.max(0, Ints.checkedCast(maxWorkers));
return (long) maxSimultaneousInputChannelsPerProcessor * frameSize;
}
/**
* Computes the amount of memory needed to read a single partition from a given number of workers.
* Number of input partitions across all {@link StageInputSlice}.
*/
static long memoryNeededForInputChannels(final int numInputWorkers)
static int computeNumInputPartitions(final List<InputSlice> inputSlices)
{
// Workers that read sorted inputs must open all channels at once to do an N-way merge. Calculate memory needs.
// Requirement: one input frame per worker, one buffered output frame.
return (long) STANDARD_FRAME_SIZE * (numInputWorkers + 1);
}
int retVal = 0;
/**
* Maximum number of workers that may exist in the current JVM.
*/
private static int computeNumWorkersInJvm(final Injector injector)
{
final AppenderatorsManager appenderatorsManager = injector.getInstance(AppenderatorsManager.class);
if (appenderatorsManager instanceof UnifiedIndexerAppenderatorsManager) {
// CliIndexer
return injector.getInstance(WorkerConfig.class).getCapacity();
} else {
// CliPeon
return 1;
for (final StageInputSlice slice : InputSlices.allStageSlices(inputSlices)) {
retVal += Iterables.size(slice.getPartitions());
}
return retVal;
}
/**
* Maximum number of concurrent processors that exist in the current JVM.
*/
private static int computeNumProcessorsInJvm(final Injector injector)
{
return injector.getInstance(Bouncer.class).getMaxCount();
}
/**
* Compute the memory allocated to each worker. Includes anything that exists outside of processing bundles.
* Maximum number of input channels that a processor may have open at once, given the provided worker assignment.
*
* Today, we only look at one thing: the amount of memory taken up by
* {@link org.apache.druid.msq.statistics.ClusterByStatisticsCollector}. This is the single largest source of memory
* usage outside processing bundles.
* To compute this, we take the maximum number of workers associated with some partition for each slice. Then we sum
* those maxes up for all broadcast slices, and for all non-broadcast slices, and take the max between those two.
* The idea is that processors first read broadcast data, then read non-broadcast data, and during both phases
* they should have at most one partition open from each slice at once.
*
* @param inputSlices object from {@link WorkOrder#getInputs()}
* @param broadcastInputNumbers object from {@link StageDefinition#getBroadcastInputNumbers()}
*/
private static long memoryPerWorker(
final long usableMemoryInJvm,
final int numWorkersInJvm
static int computeMaxSimultaneousInputChannelsPerProcessor(
final List<InputSlice> inputSlices,
final IntSet broadcastInputNumbers
)
{
final long memoryForWorkers = (long) Math.min(
usableMemoryInJvm * PARTITION_STATS_MEMORY_MAX_FRACTION,
numWorkersInJvm * PARTITION_STATS_MEMORY_MAX_BYTES
);
long totalNonBroadcastInputChannels = 0;
long totalBroadcastInputChannels = 0;
return memoryForWorkers / numWorkersInJvm;
}
final List<StageInputSlice> allStageSlices = InputSlices.allStageSlices(inputSlices);
/**
* Compute the memory allocated to each processing bundle. Any computation changes done to this method should also be
* done in its corresponding method {@link WorkerMemoryParameters#estimateUsableMemory}
*/
private static long memoryPerBundle(
final long usableMemoryInJvm,
final int numWorkersInJvm,
final int numProcessingThreadsInJvm
)
{
// One bundle per worker + one per processor. The worker bundles are used for sorting (SuperSorter) and the
// processing bundles are used for reading input and doing per-partition processing.
final int bundleCount = numWorkersInJvm + numProcessingThreadsInJvm;
for (int inputNumber = 0; inputNumber < allStageSlices.size(); inputNumber++) {
final StageInputSlice slice = allStageSlices.get(inputNumber);
// Need to subtract memoryForWorkers off the top of usableMemoryInJvm, since this is reserved for
// statistics collection.
final long memoryForWorkers = numWorkersInJvm * memoryPerWorker(usableMemoryInJvm, numWorkersInJvm);
final long memoryForBundles = usableMemoryInJvm - memoryForWorkers;
int maxWorkers = 0;
for (final ReadablePartition partition : slice.getPartitions()) {
maxWorkers = Math.max(maxWorkers, partition.getWorkerNumbers().size());
}
// Divide up the usable memory per bundle.
return memoryForBundles / bundleCount;
}
/**
* Used for estimating the usable memory for better exception messages when {@link NotEnoughMemoryFault} is thrown.
*/
private static long estimateUsableMemory(
final int numWorkersInJvm,
final int numProcessingThreadsInJvm,
final long estimatedEachBundleMemory,
final int maxConcurrentStages
)
{
final int bundleCount = numWorkersInJvm + numProcessingThreadsInJvm;
return estimateUsableMemory(numWorkersInJvm, estimatedEachBundleMemory * bundleCount, maxConcurrentStages);
}
/**
* Add overheads to the estimated bundle memoery for all the workers. Checkout {@link WorkerMemoryParameters#memoryPerWorker(long, int)}
* for the overhead calculation outside the processing bundles.
*/
private static long estimateUsableMemory(
final int numWorkersInJvm,
final long estimatedTotalBundleMemory,
final int maxConcurrentStages
)
{
// Currently, we only add the partition stats overhead since it will be the single largest overhead per worker.
final long estimateStatOverHeadPerWorker = PARTITION_STATS_MEMORY_MAX_BYTES;
final long requiredUsableMemory = estimatedTotalBundleMemory + (estimateStatOverHeadPerWorker * numWorkersInJvm);
return requiredUsableMemory * maxConcurrentStages;
}
private static long memoryNeededForHashPartitioning(final int numOutputPartitions)
{
// One standard frame for each processor output.
// May be zero, since numOutputPartitions is zero if not using hash partitioning.
return (long) STANDARD_FRAME_SIZE * numOutputPartitions;
}
/**
* Amount of heap memory available for our usage. Any computation changes done to this method should also be done in
* its corresponding method {@link WorkerMemoryParameters#calculateSuggestedMinMemoryFromUsableMemory}
*/
private static long computeUsableMemoryInJvm(final long maxMemory, final long totalLookupFootprint)
{
// Always report at least one byte, to simplify the math in createInstance.
return Math.max(
1,
(long) ((maxMemory - totalLookupFootprint) * USABLE_MEMORY_FRACTION)
);
}
/**
* Estimate amount of heap memory for the given workload to use in case usable memory is provided. This method is used
* for better exception messages when {@link NotEnoughMemoryFault} is thrown.
*/
private static long calculateSuggestedMinMemoryFromUsableMemory(long usuableMemeory, final long totalLookupFootprint)
{
return (long) ((usuableMemeory / USABLE_MEMORY_FRACTION) + totalLookupFootprint);
}
/**
* Total estimated lookup footprint. Obtained by calling {@link LookupExtractor#estimateHeapFootprint()} on
* all available lookups.
*/
private static long computeTotalLookupFootprint(final Injector injector)
{
// Subtract memory taken up by lookups. Correctness of this operation depends on lookups being loaded *before*
// we create this instance. Luckily, this is the typical mode of operation, since by default
// druid.lookup.enableLookupSyncOnStartup = true.
final LookupExtractorFactoryContainerProvider lookupManager =
injector.getInstance(LookupExtractorFactoryContainerProvider.class);
int lookupCount = 0;
long lookupFootprint = 0;
for (final String lookupName : lookupManager.getAllLookupNames()) {
final LookupExtractorFactoryContainer container = lookupManager.get(lookupName).orElse(null);
if (container != null) {
try {
final LookupExtractor extractor = container.getLookupExtractorFactory().get();
lookupFootprint += extractor.estimateHeapFootprint();
lookupCount++;
}
catch (Exception e) {
log.noStackTrace().warn(e, "Failed to load lookup [%s] for size estimation. Skipping.", lookupName);
}
if (broadcastInputNumbers.contains(inputNumber)) {
totalBroadcastInputChannels += maxWorkers;
} else {
totalNonBroadcastInputChannels += maxWorkers;
}
}
log.debug("Lookup footprint: %d lookups with %,d total bytes.", lookupCount, lookupFootprint);
return Ints.checkedCast(Math.max(totalBroadcastInputChannels, totalNonBroadcastInputChannels));
}
return lookupFootprint;
/**
* Distinct number of input workers.
*/
static int computeNumInputWorkers(final List<InputSlice> inputSlices)
{
final IntSet workerNumbers = new IntOpenHashSet();
for (final StageInputSlice slice : InputSlices.allStageSlices(inputSlices)) {
for (final ReadablePartition partition : slice.getPartitions()) {
workerNumbers.addAll(partition.getWorkerNumbers());
}
}
return workerNumbers.size();
}
/**
* Maximum number of output channels for a shuffle spec, or 0 if not knowable in advance.
*/
static int computeMaxOutputPartitions(@Nullable final ShuffleSpec shuffleSpec)
{
if (shuffleSpec == null) {
return 0;
} else {
switch (shuffleSpec.kind()) {
case HASH:
case HASH_LOCAL_SORT:
case MIX:
return shuffleSpec.partitionCount();
case GLOBAL_SORT:
if (shuffleSpec instanceof GlobalSortMaxCountShuffleSpec) {
return ((GlobalSortMaxCountShuffleSpec) shuffleSpec).getMaxPartitions();
}
// Fall through
default:
return 0;
}
}
}
/**
* Maximum number of output channels for a shuffle spec, or 0 if not knowable in advance.
*/
static int computeFramesPerOutputChannel(final OutputChannelMode outputChannelMode)
{
// If durable storage is enabled, we need one extra frame per output channel.
return outputChannelMode.isDurable() ? 2 : 1;
}
/**
* Minimum number of bytes for a bundle's free memory allotment. This must be enough to reasonably produce and
* persist an {@link IncrementalIndex}, or to run a {@link SuperSorter} with 1 thread and 2 frames.
*/
static long computeMinimumBundleFreeMemory(final int frameSize, final int numFramesPerOutputChannel)
{
// Some for partition statistics.
long minMemory = PARTITION_STATS_MIN_MEMORY_PER_BUNDLE;
// Some for a minimally-sized super-sorter.
minMemory += (long) (2 + numFramesPerOutputChannel) * frameSize;
// That's enough. Don't consider the possibility that the bundle may be used for producing IncrementalIndex,
// because PARTITION_STATS_MIN_MEMORY_PER_BUNDLE more or less covers that.
return minMemory;
}
}

View File

@ -22,13 +22,15 @@ package org.apache.druid.msq.guice;
import com.google.inject.Binder;
import com.google.inject.Provides;
import org.apache.druid.discovery.NodeRole;
import org.apache.druid.frame.processor.Bouncer;
import org.apache.druid.guice.LazySingleton;
import org.apache.druid.guice.ManageLifecycle;
import org.apache.druid.guice.annotations.LoadScope;
import org.apache.druid.indexing.worker.config.WorkerConfig;
import org.apache.druid.initialization.DruidModule;
import org.apache.druid.msq.exec.MemoryIntrospector;
import org.apache.druid.msq.exec.MemoryIntrospectorImpl;
import org.apache.druid.msq.exec.ProcessingBuffersProvider;
import org.apache.druid.msq.indexing.IndexerProcessingBuffersProvider;
import org.apache.druid.query.DruidProcessingConfig;
import org.apache.druid.query.lookup.LookupExtractorFactoryContainerProvider;
import org.apache.druid.utils.JvmUtils;
@ -42,37 +44,51 @@ import org.apache.druid.utils.JvmUtils;
public class IndexerMemoryManagementModule implements DruidModule
{
/**
* Allocate up to 75% of memory for MSQ-related stuff (if all running tasks are MSQ tasks).
* Allocate up to 60% of memory for the MSQ framework (if all running tasks are MSQ tasks). This does not include the
* memory allocated to {@link #PROCESSING_MEMORY_FRACTION}.
*/
private static final double USABLE_MEMORY_FRACTION = 0.75;
private static final double MSQ_MEMORY_FRACTION = 0.60;
/**
* Allocate up to 15% of memory for processing buffers for MSQ tasks.
*/
private static final double PROCESSING_MEMORY_FRACTION = 0.15;
@Override
public void configure(Binder binder)
{
// Nothing to do.
TaskMemoryManagementConfig.bind(binder);
}
@Provides
@LazySingleton
public Bouncer makeProcessorBouncer(final DruidProcessingConfig processingConfig)
{
return new Bouncer(processingConfig.getNumThreads());
}
@Provides
@LazySingleton
@ManageLifecycle
public MemoryIntrospector createMemoryIntrospector(
final LookupExtractorFactoryContainerProvider lookupProvider,
final TaskMemoryManagementConfig taskMemoryManagementConfig,
final DruidProcessingConfig processingConfig,
final WorkerConfig workerConfig
)
{
return new MemoryIntrospectorImpl(
lookupProvider,
JvmUtils.getRuntimeInfo().getMaxHeapSizeBytes(),
USABLE_MEMORY_FRACTION,
MSQ_MEMORY_FRACTION,
workerConfig.getCapacity(),
processingConfig.getNumThreads()
PeonMemoryManagementModule.getNumThreads(taskMemoryManagementConfig, processingConfig),
lookupProvider
);
}
@Provides
@LazySingleton
public ProcessingBuffersProvider createProcessingBuffersProvider(
final MemoryIntrospector memoryIntrospector,
final WorkerConfig workerConfig
)
{
return new IndexerProcessingBuffersProvider(
(long) (JvmUtils.getRuntimeInfo().getMaxHeapSizeBytes() * PROCESSING_MEMORY_FRACTION),
workerConfig.getCapacity(),
memoryIntrospector.numProcessingThreads()
);
}
}

View File

@ -21,22 +21,30 @@ package org.apache.druid.msq.guice;
import com.google.inject.Binder;
import com.google.inject.Provides;
import org.apache.druid.collections.NonBlockingPool;
import org.apache.druid.discovery.NodeRole;
import org.apache.druid.frame.processor.Bouncer;
import org.apache.druid.guice.LazySingleton;
import org.apache.druid.guice.annotations.Global;
import org.apache.druid.guice.annotations.LoadScope;
import org.apache.druid.initialization.DruidModule;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.msq.exec.MemoryIntrospector;
import org.apache.druid.msq.exec.MemoryIntrospectorImpl;
import org.apache.druid.msq.exec.ProcessingBuffersProvider;
import org.apache.druid.msq.indexing.PeonProcessingBuffersProvider;
import org.apache.druid.query.DruidProcessingConfig;
import org.apache.druid.query.lookup.LookupExtractorFactoryContainerProvider;
import org.apache.druid.utils.JvmUtils;
import java.nio.ByteBuffer;
/**
* Provides {@link MemoryIntrospector} for single-task-per-JVM model.
*
* @see IndexerMemoryManagementModule for multi-task-per-JVM model used on {@link org.apache.druid.cli.CliIndexer}
*/
@LoadScope(roles = NodeRole.PEON_JSON_NAME)
public class PeonMemoryManagementModule implements DruidModule
{
/**
@ -45,41 +53,61 @@ public class PeonMemoryManagementModule implements DruidModule
private static final int NUM_WORKERS_IN_JVM = 1;
/**
* Peons may have more than one processing thread, but we currently only use one of them.
*/
private static final int NUM_PROCESSING_THREADS = 1;
/**
* Allocate 75% of memory for MSQ-related stuff.
* Allocate 75% of memory for the MSQ framework.
*/
private static final double USABLE_MEMORY_FRACTION = 0.75;
@Override
public void configure(Binder binder)
{
// Nothing to do.
}
@Provides
@LazySingleton
public Bouncer makeProcessorBouncer()
{
return new Bouncer(NUM_PROCESSING_THREADS);
TaskMemoryManagementConfig.bind(binder);
}
@Provides
@LazySingleton
public MemoryIntrospector createMemoryIntrospector(
final LookupExtractorFactoryContainerProvider lookupProvider,
final Bouncer bouncer
final DruidProcessingConfig processingConfig,
final TaskMemoryManagementConfig taskMemoryManagementConfig
)
{
return new MemoryIntrospectorImpl(
lookupProvider,
JvmUtils.getRuntimeInfo().getMaxHeapSizeBytes(),
USABLE_MEMORY_FRACTION,
NUM_WORKERS_IN_JVM,
bouncer.getMaxCount()
getNumThreads(taskMemoryManagementConfig, processingConfig),
lookupProvider
);
}
@Provides
@LazySingleton
public ProcessingBuffersProvider createProcessingBuffersProvider(
@Global final NonBlockingPool<ByteBuffer> processingPool,
final MemoryIntrospector memoryIntrospector
)
{
return new PeonProcessingBuffersProvider(
processingPool,
memoryIntrospector.numProcessingThreads()
);
}
public static int getNumThreads(
final TaskMemoryManagementConfig taskMemoryManagementConfig,
final DruidProcessingConfig processingConfig
)
{
if (taskMemoryManagementConfig.getMaxThreads() == TaskMemoryManagementConfig.UNLIMITED) {
return processingConfig.getNumThreads();
} else if (taskMemoryManagementConfig.getMaxThreads() > 0) {
return Math.min(taskMemoryManagementConfig.getMaxThreads(), processingConfig.getNumThreads());
} else {
throw new IAE(
"Invalid value of %s.maxThreads[%d]",
TaskMemoryManagementConfig.BASE_PROPERTY,
taskMemoryManagementConfig.getMaxThreads()
);
}
}
}

View File

@ -0,0 +1,51 @@
/*
* 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.guice;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.inject.Binder;
import org.apache.druid.guice.JsonConfigProvider;
import org.apache.druid.java.util.common.StringUtils;
/**
* Server configuration for {@link PeonMemoryManagementModule} and {@link IndexerMemoryManagementModule}.
*/
public class TaskMemoryManagementConfig
{
public static final String BASE_PROPERTY = StringUtils.format("%s.task.memory", MSQIndexingModule.BASE_MSQ_KEY);
public static final int UNLIMITED = -1;
@JsonProperty("maxThreads")
private int maxThreads = 1;
public static void bind(final Binder binder)
{
JsonConfigProvider.bind(
binder,
BASE_PROPERTY,
TaskMemoryManagementConfig.class
);
}
public int getMaxThreads()
{
return maxThreads;
}
}

View File

@ -20,9 +20,11 @@
package org.apache.druid.msq.indexing;
import com.fasterxml.jackson.databind.ObjectMapper;
import org.apache.druid.frame.processor.Bouncer;
import org.apache.druid.collections.ResourceHolder;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.msq.exec.DataServerQueryHandlerFactory;
import org.apache.druid.msq.exec.ProcessingBuffers;
import org.apache.druid.msq.exec.WorkerMemoryParameters;
import org.apache.druid.msq.exec.WorkerStorageParameters;
import org.apache.druid.msq.kernel.FrameContext;
@ -35,6 +37,7 @@ import org.apache.druid.segment.SegmentWrangler;
import org.apache.druid.segment.incremental.RowIngestionMeters;
import org.apache.druid.segment.loading.DataSegmentPusher;
import javax.annotation.Nullable;
import java.io.File;
public class IndexerFrameContext implements FrameContext
@ -43,6 +46,8 @@ public class IndexerFrameContext implements FrameContext
private final IndexerWorkerContext context;
private final IndexIO indexIO;
private final DataSegmentProvider dataSegmentProvider;
@Nullable
private final ResourceHolder<ProcessingBuffers> processingBuffers;
private final WorkerMemoryParameters memoryParameters;
private final WorkerStorageParameters storageParameters;
private final DataServerQueryHandlerFactory dataServerQueryHandlerFactory;
@ -52,6 +57,7 @@ public class IndexerFrameContext implements FrameContext
IndexerWorkerContext context,
IndexIO indexIO,
DataSegmentProvider dataSegmentProvider,
@Nullable ResourceHolder<ProcessingBuffers> processingBuffers,
DataServerQueryHandlerFactory dataServerQueryHandlerFactory,
WorkerMemoryParameters memoryParameters,
WorkerStorageParameters storageParameters
@ -61,6 +67,7 @@ public class IndexerFrameContext implements FrameContext
this.context = context;
this.indexIO = indexIO;
this.dataSegmentProvider = dataSegmentProvider;
this.processingBuffers = processingBuffers;
this.memoryParameters = memoryParameters;
this.storageParameters = storageParameters;
this.dataServerQueryHandlerFactory = dataServerQueryHandlerFactory;
@ -135,15 +142,19 @@ public class IndexerFrameContext implements FrameContext
}
@Override
public WorkerMemoryParameters memoryParameters()
public ProcessingBuffers processingBuffers()
{
return memoryParameters;
if (processingBuffers != null) {
return processingBuffers.get();
} else {
throw new ISE("No processing buffers");
}
}
@Override
public Bouncer processorBouncer()
public WorkerMemoryParameters memoryParameters()
{
return context.injector().getInstance(Bouncer.class);
return memoryParameters;
}
@Override
@ -155,6 +166,8 @@ public class IndexerFrameContext implements FrameContext
@Override
public void close()
{
// Nothing to close.
if (processingBuffers != null) {
processingBuffers.close();
}
}
}

View File

@ -0,0 +1,89 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.msq.indexing;
import org.apache.druid.cli.CliIndexer;
import org.apache.druid.collections.ReferenceCountingResourceHolder;
import org.apache.druid.collections.ResourceHolder;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.msq.exec.ProcessingBuffersProvider;
import org.apache.druid.msq.exec.ProcessingBuffersSet;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.List;
/**
* Production implementation of {@link ProcessingBuffersProvider} for tasks in {@link CliIndexer}.
*/
public class IndexerProcessingBuffersProvider implements ProcessingBuffersProvider
{
private static final int MIN_BUFFER_SIZE = 1_000_000;
private final long heapMemoryToUse;
private final int taskCapacity;
private final int numThreads;
public IndexerProcessingBuffersProvider(final long heapMemoryToUse, final int taskCapacity, final int numThreads)
{
this.heapMemoryToUse = heapMemoryToUse;
this.taskCapacity = taskCapacity;
this.numThreads = numThreads;
}
@Override
public ResourceHolder<ProcessingBuffersSet> acquire(int poolSize)
{
if (poolSize == 0) {
return new ReferenceCountingResourceHolder<>(ProcessingBuffersSet.EMPTY, () -> {});
}
final long heapMemoryPerWorker = heapMemoryToUse / taskCapacity;
final int numThreadsPerWorker = (int) Math.min(
numThreads,
heapMemoryPerWorker / MIN_BUFFER_SIZE
);
if (numThreadsPerWorker < 1) {
// Should not happen unless the CliIndexer has an unreasonable configuration.
// CliIndexer typically has well in excess of 1 MB (min buffer size) of heap per task.
throw new ISE("Cannot acquire buffers, available heap memory is not enough for task capacity[%d]", taskCapacity);
}
// bufferPools has one list per "poolSize"; each of those lists has "bufferCount" buffers of size "sliceSize".
final List<List<ByteBuffer>> bufferPools = new ArrayList<>(poolSize);
final int sliceSize = (int) Math.min(Integer.MAX_VALUE, heapMemoryPerWorker / numThreadsPerWorker);
for (int i = 0; i < poolSize; i++) {
final List<ByteBuffer> bufferPool = new ArrayList<>(numThreadsPerWorker);
bufferPools.add(bufferPool);
for (int j = 0; j < numThreadsPerWorker; j++) {
bufferPool.add(ByteBuffer.allocate(sliceSize));
}
}
// bufferPools is built, return it as a ProcessingBuffersSet.
return new ReferenceCountingResourceHolder<>(
ProcessingBuffersSet.fromCollection(bufferPools),
() -> {} // Garbage collection will reclaim the buffers, since they are on-heap
);
}
}

View File

@ -24,6 +24,7 @@ import com.google.common.annotations.VisibleForTesting;
import com.google.errorprone.annotations.concurrent.GuardedBy;
import com.google.inject.Injector;
import com.google.inject.Key;
import org.apache.druid.collections.ResourceHolder;
import org.apache.druid.guice.annotations.EscalatedGlobal;
import org.apache.druid.guice.annotations.Smile;
import org.apache.druid.indexing.common.SegmentCacheManagerFactory;
@ -34,7 +35,8 @@ import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.msq.exec.ControllerClient;
import org.apache.druid.msq.exec.DataServerQueryHandlerFactory;
import org.apache.druid.msq.exec.MemoryIntrospector;
import org.apache.druid.msq.exec.OutputChannelMode;
import org.apache.druid.msq.exec.ProcessingBuffersProvider;
import org.apache.druid.msq.exec.ProcessingBuffersSet;
import org.apache.druid.msq.exec.TaskDataSegmentProvider;
import org.apache.druid.msq.exec.Worker;
import org.apache.druid.msq.exec.WorkerClient;
@ -45,7 +47,7 @@ import org.apache.druid.msq.indexing.client.IndexerControllerClient;
import org.apache.druid.msq.indexing.client.IndexerWorkerClient;
import org.apache.druid.msq.indexing.client.WorkerChatHandler;
import org.apache.druid.msq.kernel.FrameContext;
import org.apache.druid.msq.kernel.QueryDefinition;
import org.apache.druid.msq.kernel.WorkOrder;
import org.apache.druid.msq.util.MultiStageQueryContext;
import org.apache.druid.query.QueryContext;
import org.apache.druid.query.QueryToolChestWarehouse;
@ -79,12 +81,16 @@ public class IndexerWorkerContext implements WorkerContext
private final DataServerQueryHandlerFactory dataServerQueryHandlerFactory;
private final ServiceClientFactory clientFactory;
private final MemoryIntrospector memoryIntrospector;
private final ProcessingBuffersProvider processingBuffersProvider;
private final int maxConcurrentStages;
private final boolean includeAllCounters;
@GuardedBy("this")
private ServiceLocator controllerLocator;
// Written under synchronized(this) using double-checked locking.
private volatile ResourceHolder<ProcessingBuffersSet> processingBuffersSet;
public IndexerWorkerContext(
final MSQWorkerTask task,
final TaskToolbox toolbox,
@ -94,6 +100,7 @@ public class IndexerWorkerContext implements WorkerContext
final TaskDataSegmentProvider dataSegmentProvider,
final ServiceClientFactory clientFactory,
final MemoryIntrospector memoryIntrospector,
final ProcessingBuffersProvider processingBuffersProvider,
final DataServerQueryHandlerFactory dataServerQueryHandlerFactory
)
{
@ -105,6 +112,7 @@ public class IndexerWorkerContext implements WorkerContext
this.dataSegmentProvider = dataSegmentProvider;
this.clientFactory = clientFactory;
this.memoryIntrospector = memoryIntrospector;
this.processingBuffersProvider = processingBuffersProvider;
this.dataServerQueryHandlerFactory = dataServerQueryHandlerFactory;
final QueryContext queryContext = QueryContext.of(task.getContext());
@ -127,6 +135,7 @@ public class IndexerWorkerContext implements WorkerContext
final MemoryIntrospector memoryIntrospector = injector.getInstance(MemoryIntrospector.class);
final OverlordClient overlordClient =
injector.getInstance(OverlordClient.class).withRetryPolicy(StandardRetryPolicy.unlimited());
final ProcessingBuffersProvider processingBuffersProvider = injector.getInstance(ProcessingBuffersProvider.class);
final ObjectMapper smileMapper = injector.getInstance(Key.get(ObjectMapper.class, Smile.class));
final QueryToolChestWarehouse warehouse = injector.getInstance(QueryToolChestWarehouse.class);
@ -139,6 +148,7 @@ public class IndexerWorkerContext implements WorkerContext
new TaskDataSegmentProvider(toolbox.getCoordinatorClient(), segmentCacheManager, indexIO),
serviceClientFactory,
memoryIntrospector,
processingBuffersProvider,
new DataServerQueryHandlerFactory(
toolbox.getCoordinatorClient(),
serviceClientFactory,
@ -191,6 +201,14 @@ public class IndexerWorkerContext implements WorkerContext
}
}
});
closer.register(() -> {
synchronized (this) {
if (processingBuffersSet != null) {
processingBuffersSet.close();
processingBuffersSet = null;
}
}
});
// Register the periodic controller checker
final ExecutorService periodicControllerCheckerExec = Execs.singleThreaded("controller-status-checker-%s");
@ -281,23 +299,39 @@ public class IndexerWorkerContext implements WorkerContext
}
@Override
public FrameContext frameContext(QueryDefinition queryDef, int stageNumber, OutputChannelMode outputChannelMode)
public FrameContext frameContext(WorkOrder workOrder)
{
if (processingBuffersSet == null) {
synchronized (this) {
if (processingBuffersSet == null) {
processingBuffersSet = processingBuffersProvider.acquire(
workOrder.getQueryDefinition(),
maxConcurrentStages()
);
}
}
}
final WorkerMemoryParameters memoryParameters =
WorkerMemoryParameters.createProductionInstance(workOrder, memoryIntrospector, maxConcurrentStages);
log.info("Memory parameters for stage[%s]: %s", workOrder.getStageDefinition().getId(), memoryParameters);
return new IndexerFrameContext(
queryDef.getStageDefinition(stageNumber).getId(),
workOrder.getStageDefinition().getId(),
this,
indexIO,
dataSegmentProvider,
processingBuffersSet.get().acquireForStage(workOrder.getStageDefinition()),
dataServerQueryHandlerFactory,
WorkerMemoryParameters.createProductionInstanceForWorker(injector, queryDef, stageNumber, maxConcurrentStages),
WorkerStorageParameters.createProductionInstance(injector, outputChannelMode)
memoryParameters,
WorkerStorageParameters.createProductionInstance(injector, workOrder.getOutputChannelMode())
);
}
@Override
public int threadCount()
{
return memoryIntrospector.numProcessorsInJvm();
return memoryIntrospector.numProcessingThreads();
}
@Override

View File

@ -0,0 +1,98 @@
/*
* 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.indexing;
import org.apache.druid.cli.CliPeon;
import org.apache.druid.collections.NonBlockingPool;
import org.apache.druid.collections.ReferenceCountingResourceHolder;
import org.apache.druid.collections.ResourceHolder;
import org.apache.druid.error.DruidException;
import org.apache.druid.java.util.common.io.Closer;
import org.apache.druid.msq.exec.ProcessingBuffersProvider;
import org.apache.druid.msq.exec.ProcessingBuffersSet;
import org.apache.druid.utils.CloseableUtils;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.List;
import java.util.concurrent.atomic.AtomicBoolean;
/**
* Production implementation of {@link ProcessingBuffersProvider} for tasks in {@link CliPeon}.
*/
public class PeonProcessingBuffersProvider implements ProcessingBuffersProvider
{
private final AtomicBoolean acquired = new AtomicBoolean(false);
private final NonBlockingPool<ByteBuffer> bufferPool;
private final int bufferCount;
public PeonProcessingBuffersProvider(
final NonBlockingPool<ByteBuffer> bufferPool,
final int bufferCount
)
{
this.bufferPool = bufferPool;
this.bufferCount = bufferCount;
}
@Override
public ResourceHolder<ProcessingBuffersSet> acquire(int poolSize)
{
if (poolSize == 0) {
return new ReferenceCountingResourceHolder<>(ProcessingBuffersSet.EMPTY, () -> {});
}
if (!acquired.compareAndSet(false, true)) {
// We expect a single task in the JVM for CliPeon.
throw DruidException.defensive("Expected a single call to acquire() for[%s]", getClass().getName());
}
final Closer closer = Closer.create();
try {
// bufferPools has one list per "poolSize"; each of those lists has "bufferCount" buffers.
// Build these by acquiring "bufferCount" processing buffers and slicing each one up into "poolSize" slices.
final List<List<ByteBuffer>> bufferPools = new ArrayList<>();
for (int i = 0; i < poolSize; i++) {
bufferPools.add(new ArrayList<>(bufferCount));
}
for (int i = 0; i < bufferCount; i++) {
final ResourceHolder<ByteBuffer> bufferHolder = closer.register(bufferPool.take());
final ByteBuffer buffer = bufferHolder.get().duplicate();
final int sliceSize = buffer.capacity() / poolSize;
for (int j = 0; j < poolSize; j++) {
buffer.position(sliceSize * j).limit(sliceSize * (j + 1));
bufferPools.get(j).add(buffer.slice());
}
}
// bufferPools is built, return it as a ProcessingBuffersSet.
return new ReferenceCountingResourceHolder<>(
ProcessingBuffersSet.fromCollection(bufferPools),
closer
);
}
catch (Throwable e) {
throw CloseableUtils.closeAndWrapInCatch(e, closer);
}
}
}

View File

@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonInclude;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.annotation.JsonTypeName;
import org.apache.druid.java.util.common.StringUtils;
import java.util.Objects;
@ -36,6 +37,7 @@ public class NotEnoughMemoryFault extends BaseMSQFault
private final long usableMemory;
private final int serverWorkers;
private final int serverThreads;
private final int inputWorkers;
private final int maxConcurrentStages;
@JsonCreator
@ -45,22 +47,33 @@ public class NotEnoughMemoryFault extends BaseMSQFault
@JsonProperty("usableMemory") final long usableMemory,
@JsonProperty("serverWorkers") final int serverWorkers,
@JsonProperty("serverThreads") final int serverThreads,
@JsonProperty("inputWorkers") final int inputWorkers,
@JsonProperty("maxConcurrentStages") final int maxConcurrentStages
)
{
super(
CODE,
"Not enough memory. Required at least %,d bytes. (total = %,d bytes; usable = %,d bytes; "
+ "worker capacity = %,d; processing threads = %,d; concurrent stages = %,d). "
"Not enough memory. "
+ (suggestedServerMemory > 0
? StringUtils.format("Minimum bytes[%,d] is needed for the current configuration. ", suggestedServerMemory)
: "")
+ "(total bytes[%,d]; "
+ "usable bytes[%,d]; "
+ "input workers[%,d]; "
+ "concurrent stages[%,d]; "
+ "server worker capacity[%,d]; "
+ "server processing threads[%,d]). "
+ "Increase JVM memory with the -Xmx option"
+ (inputWorkers > 1 ? ", or reduce maxNumTasks for this query" : "")
+ (maxConcurrentStages > 1 ? ", or reduce maxConcurrentStages for this query" : "")
+ (serverWorkers > 1 ? ", or reduce worker capacity on this server" : "")
+ (maxConcurrentStages > 1 ? ", or reduce maxConcurrentStages for this query" : ""),
suggestedServerMemory,
+ (serverThreads > 1 ? ", or reduce processing threads on this server" : ""),
serverMemory,
usableMemory,
inputWorkers,
maxConcurrentStages,
serverWorkers,
serverThreads,
maxConcurrentStages
serverThreads
);
this.suggestedServerMemory = suggestedServerMemory;
@ -68,10 +81,12 @@ public class NotEnoughMemoryFault extends BaseMSQFault
this.usableMemory = usableMemory;
this.serverWorkers = serverWorkers;
this.serverThreads = serverThreads;
this.inputWorkers = inputWorkers;
this.maxConcurrentStages = maxConcurrentStages;
}
@JsonProperty
@JsonInclude(JsonInclude.Include.NON_DEFAULT)
public long getSuggestedServerMemory()
{
return suggestedServerMemory;
@ -101,6 +116,13 @@ public class NotEnoughMemoryFault extends BaseMSQFault
return serverThreads;
}
@JsonProperty
@JsonInclude(JsonInclude.Include.NON_DEFAULT)
public int getInputWorkers()
{
return inputWorkers;
}
@JsonProperty
@JsonInclude(JsonInclude.Include.NON_DEFAULT)
public int getMaxConcurrentStages()
@ -126,6 +148,7 @@ public class NotEnoughMemoryFault extends BaseMSQFault
&& usableMemory == that.usableMemory
&& serverWorkers == that.serverWorkers
&& serverThreads == that.serverThreads
&& inputWorkers == that.inputWorkers
&& maxConcurrentStages == that.maxConcurrentStages;
}
@ -139,6 +162,7 @@ public class NotEnoughMemoryFault extends BaseMSQFault
usableMemory,
serverWorkers,
serverThreads,
inputWorkers,
maxConcurrentStages
);
}
@ -148,10 +172,11 @@ public class NotEnoughMemoryFault extends BaseMSQFault
{
return "NotEnoughMemoryFault{" +
"suggestedServerMemory=" + suggestedServerMemory +
" bytes, serverMemory=" + serverMemory +
" bytes, usableMemory=" + usableMemory +
" bytes, serverWorkers=" + serverWorkers +
", serverMemory=" + serverMemory +
", usableMemory=" + usableMemory +
", serverWorkers=" + serverWorkers +
", serverThreads=" + serverThreads +
", inputWorkers=" + inputWorkers +
", maxConcurrentStages=" + maxConcurrentStages +
'}';
}

View File

@ -44,9 +44,8 @@ public class TooManyRowsWithSameKeyFault extends BaseMSQFault
{
super(
CODE,
"Too many rows with the same key[%s] during sort-merge join (bytes buffered[%,d], limit[%,d]). "
+ "Try increasing heap memory available to workers, "
+ "or adjusting your query to process fewer rows with this key.",
"Too many rows with the same key[%s] on both sides of sort-merge join (bytes buffered[%,d], limit[%,d]). "
+ "Try adjusting your query such that there are fewer rows with this key on at least one side of the join.",
key,
numBytes,
maxBytes

View File

@ -28,6 +28,7 @@ import com.google.common.base.Preconditions;
import com.google.common.collect.Iterables;
import org.apache.druid.frame.processor.OutputChannelFactory;
import org.apache.druid.frame.processor.OutputChannels;
import org.apache.druid.frame.processor.manager.ConcurrencyLimitedProcessorManager;
import org.apache.druid.frame.processor.manager.ProcessorManagers;
import org.apache.druid.indexer.partitions.DynamicPartitionsSpec;
import org.apache.druid.indexer.partitions.PartitionsSpec;
@ -210,21 +211,29 @@ public class SegmentGeneratorFrameProcessorFactory
);
return new ProcessorsAndChannels<>(
ProcessorManagers.of(workers)
.withAccumulation(
new HashSet<>(),
(acc, segment) -> {
if (segment != null) {
acc.add(segment);
}
// Run at most one segmentGenerator per work order, since segment generation memory is carved out
// per-worker, not per-processor. See WorkerMemoryParameters for how the memory limits are calculated.
new ConcurrencyLimitedProcessorManager<>(ProcessorManagers.of(workers), 1)
.withAccumulation(
new HashSet<>(),
(acc, segment) -> {
if (segment != null) {
acc.add(segment);
}
return acc;
}
),
return acc;
}
),
OutputChannels.none()
);
}
@Override
public boolean usesProcessingBuffers()
{
return false;
}
@Override
public TypeReference<Set<DataSegment>> getResultTypeReference()
{

View File

@ -35,6 +35,9 @@ public class InputSpecs
// No instantiation.
}
/**
* Returns the set of input stages, from {@link StageInputSpec}, for a given list of {@link InputSpec}.
*/
public static IntSet getStageNumbers(final List<InputSpec> specs)
{
final IntSet retVal = new IntRBTreeSet();

View File

@ -20,9 +20,9 @@
package org.apache.druid.msq.kernel;
import com.fasterxml.jackson.databind.ObjectMapper;
import org.apache.druid.frame.processor.Bouncer;
import org.apache.druid.msq.exec.DataServerQueryHandlerFactory;
import org.apache.druid.msq.exec.OutputChannelMode;
import org.apache.druid.msq.exec.ProcessingBuffers;
import org.apache.druid.msq.exec.WorkerImpl;
import org.apache.druid.msq.exec.WorkerMemoryParameters;
import org.apache.druid.msq.exec.WorkerStorageParameters;
import org.apache.druid.msq.querykit.DataSegmentProvider;
@ -40,7 +40,7 @@ import java.io.File;
* Provides services and objects for the functioning of the frame processors. Scoped to a specific stage of a
* specific query, i.e., one {@link WorkOrder}.
*
* Generated by {@link org.apache.druid.msq.exec.WorkerContext#frameContext(QueryDefinition, int, OutputChannelMode)}.
* Generated by {@link org.apache.druid.msq.exec.WorkerContext#frameContext(WorkOrder)}.
*/
public interface FrameContext extends Closeable
{
@ -54,6 +54,9 @@ public interface FrameContext extends Closeable
DataServerQueryHandlerFactory dataServerQueryHandlerFactory();
/**
* Temporary directory, fully owned by this particular stage.
*/
File tempDir();
ObjectMapper jsonMapper();
@ -66,7 +69,7 @@ public interface FrameContext extends Closeable
IndexMergerV9 indexMerger();
Bouncer processorBouncer();
ProcessingBuffers processingBuffers();
WorkerMemoryParameters memoryParameters();
@ -76,4 +79,11 @@ public interface FrameContext extends Closeable
{
return new File(tempDir(), name);
}
/**
* Releases resources used in processing. This is called when processing has completed, but before results are
* cleaned up. Specifically, it is called by {@link WorkerImpl.KernelHolder#processorCloser}.
*/
@Override
void close();
}

View File

@ -78,6 +78,11 @@ public interface FrameProcessorFactory<T, R, ExtraInfoType>
boolean removeNullBytes
) throws IOException;
/**
* Whether processors from this factory use {@link org.apache.druid.msq.exec.ProcessingBuffers}.
*/
boolean usesProcessingBuffers();
@Nullable
TypeReference<R> getResultTypeReference();

View File

@ -146,6 +146,13 @@ public class StageDefinition
}
}
public static boolean mustGatherResultKeyStatistics(@Nullable final ShuffleSpec shuffleSpec)
{
return shuffleSpec != null
&& shuffleSpec.kind() == ShuffleKind.GLOBAL_SORT
&& ((GlobalSortShuffleSpec) shuffleSpec).mustGatherResultKeyStatistics();
}
public static StageDefinitionBuilder builder(final int stageNumber)
{
return new StageDefinitionBuilder(stageNumber);
@ -302,14 +309,10 @@ public class StageDefinition
* For eg: we know there's exactly one partition in query shapes like `select with limit`.
* <br></br>
* In such cases, we return a false.
*
* @return
*/
public boolean mustGatherResultKeyStatistics()
{
return shuffleSpec != null
&& shuffleSpec.kind() == ShuffleKind.GLOBAL_SORT
&& ((GlobalSortShuffleSpec) shuffleSpec).mustGatherResultKeyStatistics();
return mustGatherResultKeyStatistics(shuffleSpec);
}
public Either<Long, ClusterByPartitions> generatePartitionBoundariesForShuffle(

View File

@ -19,6 +19,8 @@
package org.apache.druid.msq.kernel.worker;
import org.apache.druid.msq.exec.ProcessingBuffers;
/**
* Phases that a stage can be in, as far as the worker is concerned.
*
@ -99,6 +101,8 @@ public enum WorkerStagePhase
/**
* Whether this phase indicates a stage is running and consuming its full complement of resources.
*
* Importantly, stages that are not running are not holding {@link ProcessingBuffers}.
*
* There are still some resources that can be consumed by stages that are not running. For example, in the
* {@link #FINISHED} state, stages can still have data on disk that has not been cleaned-up yet, some pointers
* to that data that still reside in memory, and some counters in memory available for collection by the controller.

View File

@ -352,7 +352,7 @@ public abstract class BaseLeafFrameProcessorFactory extends BaseFrameProcessorFa
return BroadcastJoinSegmentMapFnProcessor.create(
query,
broadcastInputs,
frameContext.memoryParameters().getBroadcastJoinMemory()
frameContext.memoryParameters().getBroadcastBufferMemory()
);
}
}

View File

@ -83,7 +83,7 @@ public class BroadcastJoinSegmentMapFnProcessor implements FrameProcessor<Functi
* @param channels list of input channels
* @param channelReaders list of input channel readers; corresponds one-to-one with "channels"
* @param memoryReservedForBroadcastJoin total bytes of frames we are permitted to use; derived from
* {@link WorkerMemoryParameters#getBroadcastJoinMemory()}
* {@link WorkerMemoryParameters#getBroadcastBufferMemory()}
*/
public BroadcastJoinSegmentMapFnProcessor(
final Query<?> query,

View File

@ -174,6 +174,11 @@ public class WindowOperatorQueryFrameProcessorFactory extends BaseFrameProcessor
);
}
@Override
public boolean usesProcessingBuffers()
{
return false;
}
@Override
public boolean equals(Object o)

View File

@ -140,6 +140,12 @@ public class OffsetLimitFrameProcessorFactory extends BaseFrameProcessorFactory
);
}
@Override
public boolean usesProcessingBuffers()
{
return false;
}
@Override
public boolean equals(Object o)
{

View File

@ -197,6 +197,12 @@ public class SortMergeJoinFrameProcessorFactory extends BaseFrameProcessorFactor
);
}
@Override
public boolean usesProcessingBuffers()
{
return false;
}
/**
* Extracts key columns from a {@link JoinConditionAnalysis}. The returned list has two elements: 0 is the
* left-hand side, 1 is the right-hand side. Each sub-list has one element for each equi-condition.

View File

@ -129,4 +129,10 @@ public class GroupByPostShuffleFrameProcessorFactory extends BaseFrameProcessorF
OutputChannels.wrapReadOnly(ImmutableList.copyOf(outputChannels.values()))
);
}
@Override
public boolean usesProcessingBuffers()
{
return false;
}
}

View File

@ -20,6 +20,7 @@
package org.apache.druid.msq.querykit.groupby;
import com.google.common.collect.Iterables;
import org.apache.druid.collections.NonBlockingPool;
import org.apache.druid.collections.ResourceHolder;
import org.apache.druid.frame.Frame;
import org.apache.druid.frame.channel.FrameWithPartition;
@ -60,6 +61,7 @@ import org.apache.druid.segment.column.RowSignature;
import org.apache.druid.timeline.SegmentId;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.List;
import java.util.function.Function;
@ -72,6 +74,7 @@ public class GroupByPreShuffleFrameProcessor extends BaseLeafFrameProcessor
private static final Logger log = new Logger(GroupByPreShuffleFrameProcessor.class);
private final GroupByQuery query;
private final GroupingEngine groupingEngine;
private final NonBlockingPool<ByteBuffer> bufferPool;
private final ColumnSelectorFactory frameWriterColumnSelectorFactory;
private final Closer closer = Closer.create();
@ -84,6 +87,7 @@ public class GroupByPreShuffleFrameProcessor extends BaseLeafFrameProcessor
public GroupByPreShuffleFrameProcessor(
final GroupByQuery query,
final GroupingEngine groupingEngine,
final NonBlockingPool<ByteBuffer> bufferPool,
final ReadableInput baseInput,
final Function<SegmentReference, SegmentReference> segmentMapFn,
final ResourceHolder<WritableFrameChannel> outputChannelHolder,
@ -98,6 +102,7 @@ public class GroupByPreShuffleFrameProcessor extends BaseLeafFrameProcessor
);
this.query = query;
this.groupingEngine = groupingEngine;
this.bufferPool = bufferPool;
this.frameWriterColumnSelectorFactory = RowBasedGrouperHelper.createResultRowBasedColumnSelectorFactory(
query,
() -> resultYielder.get(),
@ -155,6 +160,7 @@ public class GroupByPreShuffleFrameProcessor extends BaseLeafFrameProcessor
query.withQuerySegmentSpec(new SpecificSegmentSpec(segment.getDescriptor())),
mappedSegment.asCursorFactory(),
mappedSegment.as(TimeBoundaryInspector.class),
bufferPool,
null
);
@ -189,6 +195,7 @@ public class GroupByPreShuffleFrameProcessor extends BaseLeafFrameProcessor
query.withQuerySegmentSpec(new MultipleIntervalSegmentSpec(Intervals.ONLY_ETERNITY)),
mappedSegment.asCursorFactory(),
mappedSegment.as(TimeBoundaryInspector.class),
bufferPool,
null
);

View File

@ -65,10 +65,17 @@ public class GroupByPreShuffleFrameProcessorFactory extends BaseLeafFrameProcess
return new GroupByPreShuffleFrameProcessor(
query,
frameContext.groupingEngine(),
frameContext.processingBuffers().getBufferPool(),
baseInput,
segmentMapFn,
outputChannelHolder,
frameWriterFactoryHolder
);
}
@Override
public boolean usesProcessingBuffers()
{
return true;
}
}

View File

@ -117,6 +117,12 @@ public class ExportResultsFrameProcessorFactory implements FrameProcessorFactory
return resultsContext;
}
@Override
public boolean usesProcessingBuffers()
{
return false;
}
@Override
public ProcessorsAndChannels<Object, Object> makeProcessors(
StageDefinition stageDefinition,

View File

@ -115,4 +115,10 @@ public class QueryResultFrameProcessorFactory extends BaseFrameProcessorFactory
OutputChannels.wrapReadOnly(ImmutableList.copyOf(outputChannels.values()))
);
}
@Override
public boolean usesProcessingBuffers()
{
return false;
}
}

View File

@ -85,4 +85,10 @@ public class ScanQueryFrameProcessorFactory extends BaseLeafFrameProcessorFactor
frameWriterFactoryHolder
);
}
@Override
public boolean usesProcessingBuffers()
{
return false;
}
}

View File

@ -39,7 +39,7 @@ public class ControllerMemoryParametersTest
1
);
Assert.assertEquals(100_400_000, memoryParameters.getPartitionStatisticsMaxRetainedBytes());
Assert.assertEquals(101_400_000, memoryParameters.getPartitionStatisticsMaxRetainedBytes());
}
@Test
@ -50,7 +50,7 @@ public class ControllerMemoryParametersTest
100
);
Assert.assertEquals(103_800_000, memoryParameters.getPartitionStatisticsMaxRetainedBytes());
Assert.assertEquals(104_800_000, memoryParameters.getPartitionStatisticsMaxRetainedBytes());
}
@Test
@ -61,7 +61,7 @@ public class ControllerMemoryParametersTest
1
);
Assert.assertEquals(49_200_000, memoryParameters.getPartitionStatisticsMaxRetainedBytes());
Assert.assertEquals(50_200_000, memoryParameters.getPartitionStatisticsMaxRetainedBytes());
}
@Test
@ -91,7 +91,6 @@ public class ControllerMemoryParametersTest
Assert.assertEquals(1, fault.getServerWorkers());
Assert.assertEquals(NUM_PROCESSORS_IN_JVM, fault.getServerThreads());
Assert.assertEquals(24_000_000, fault.getUsableMemory());
Assert.assertEquals(33_750_000, fault.getSuggestedServerMemory());
}
@Test
@ -102,7 +101,7 @@ public class ControllerMemoryParametersTest
1
);
Assert.assertEquals(25_000_000, memoryParameters.getPartitionStatisticsMaxRetainedBytes());
Assert.assertEquals(26_000_000, memoryParameters.getPartitionStatisticsMaxRetainedBytes());
}
private MemoryIntrospector makeMemoryIntrospector(
@ -111,11 +110,11 @@ public class ControllerMemoryParametersTest
)
{
return new MemoryIntrospectorImpl(
new TestLookupProvider(ImmutableMap.of()),
totalMemoryInJvm,
USABLE_MEMORY_FRACTION,
numQueriesInJvm,
NUM_PROCESSORS_IN_JVM
NUM_PROCESSORS_IN_JVM,
new TestLookupProvider(ImmutableMap.of())
);
}
}

View File

@ -1461,7 +1461,7 @@ public class MSQInsertTest extends MSQTestBase
final File toRead = getResourceAsTemporaryFile("/wikipedia-sampled.json");
final String toReadFileNameAsJson = queryFramework().queryJsonMapper().writeValueAsString(toRead.getAbsolutePath());
Mockito.doReturn(500).when(workerMemoryParameters).getStandardFrameSize();
Mockito.doReturn(500).when(workerMemoryParameters).getFrameSize();
testIngestQuery().setSql(" insert into foo1 SELECT\n"
+ " floor(TIME_PARSE(\"timestamp\") to day) AS __time,\n"

View File

@ -19,185 +19,387 @@
package org.apache.druid.msq.exec;
import com.google.common.collect.ImmutableList;
import it.unimi.dsi.fastutil.ints.IntSet;
import it.unimi.dsi.fastutil.ints.IntSets;
import nl.jqno.equalsverifier.EqualsVerifier;
import org.apache.druid.frame.key.ClusterBy;
import org.apache.druid.frame.key.KeyColumn;
import org.apache.druid.frame.key.KeyOrder;
import org.apache.druid.msq.indexing.error.MSQException;
import org.apache.druid.msq.indexing.error.MSQFault;
import org.apache.druid.msq.indexing.error.NotEnoughMemoryFault;
import org.apache.druid.msq.indexing.error.TooManyWorkersFault;
import org.apache.druid.msq.input.InputSlice;
import org.apache.druid.msq.input.stage.ReadablePartitions;
import org.apache.druid.msq.input.stage.StageInputSlice;
import org.apache.druid.msq.kernel.GlobalSortTargetSizeShuffleSpec;
import org.apache.druid.msq.kernel.ShuffleSpec;
import org.junit.Assert;
import org.junit.Test;
import java.util.Arrays;
import java.util.List;
import java.util.stream.Collectors;
public class WorkerMemoryParametersTest
{
@Test
public void test_oneWorkerInJvm_alone()
public void test_1WorkerInJvm_alone_1Thread()
{
Assert.assertEquals(params(335_500_000, 1, 41, 75_000_000), create(1_000_000_000, 1, 1, 1, 1, 0, 0));
Assert.assertEquals(params(223_000_000, 2, 13, 75_000_000), create(1_000_000_000, 1, 2, 1, 1, 0, 0));
Assert.assertEquals(params(133_000_000, 4, 3, 75_000_000), create(1_000_000_000, 1, 4, 1, 1, 0, 0));
Assert.assertEquals(params(73_000_000, 3, 2, 75_000_000), create(1_000_000_000, 1, 8, 1, 1, 0, 0));
Assert.assertEquals(params(49_923_076, 2, 2, 75_000_000), create(1_000_000_000, 1, 12, 1, 1, 0, 0));
final int numThreads = 1;
final int frameSize = WorkerMemoryParameters.DEFAULT_FRAME_SIZE;
final MSQException e = Assert.assertThrows(
MSQException.class,
() -> create(1_000_000_000, 1, 32, 1, 1, 0, 0)
final MemoryIntrospectorImpl memoryIntrospector = createMemoryIntrospector(1_250_000_000, 1, numThreads);
final List<InputSlice> slices = makeInputSlices(ReadablePartitions.striped(0, 1, numThreads));
final IntSet broadcastInputs = IntSets.emptySet();
final ShuffleSpec shuffleSpec = makeSortShuffleSpec();
Assert.assertEquals(
new WorkerMemoryParameters(973_000_000, frameSize, 1, 874, 97_300_000, 0),
WorkerMemoryParameters.createInstance(memoryIntrospector, frameSize, slices, broadcastInputs, shuffleSpec, 1, 1)
);
Assert.assertEquals(new NotEnoughMemoryFault(1_588_044_000, 1_000_000_000, 750_000_000, 1, 32, 1), e.getFault());
final MSQFault fault = Assert.assertThrows(MSQException.class, () -> create(1_000_000_000, 2, 32, 1, 1, 0, 0))
.getFault();
Assert.assertEquals(new NotEnoughMemoryFault(2024045333, 1_000_000_000, 750_000_000, 2, 32, 1), fault);
}
@Test
public void test_oneWorkerInJvm_alone_twoConcurrentStages()
public void test_1WorkerInJvm_alone_withBroadcast_1Thread()
{
Assert.assertEquals(params(166_750_000, 1, 20, 37_500_000), create(1_000_000_000, 1, 1, 2, 1, 0, 0));
Assert.assertEquals(params(110_500_000, 2, 6, 37_500_000), create(1_000_000_000, 1, 2, 2, 1, 0, 0));
Assert.assertEquals(params(65_500_000, 2, 3, 37_500_000), create(1_000_000_000, 1, 4, 2, 1, 0, 0));
Assert.assertEquals(params(35_500_000, 1, 3, 37_500_000), create(1_000_000_000, 1, 8, 2, 1, 0, 0));
final int numThreads = 1;
final int frameSize = WorkerMemoryParameters.DEFAULT_FRAME_SIZE;
final MSQException e = Assert.assertThrows(
MSQException.class,
() -> create(1_000_000_000, 1, 12, 2, 1, 0, 0)
final MemoryIntrospectorImpl memoryIntrospector = createMemoryIntrospector(1_250_000_000, 1, numThreads);
final List<InputSlice> slices = makeInputSlices(
ReadablePartitions.striped(0, 1, numThreads),
ReadablePartitions.striped(0, 1, 1)
);
final IntSet broadcastInputs = IntSets.singleton(1);
final ShuffleSpec shuffleSpec = makeSortShuffleSpec();
Assert.assertEquals(new NotEnoughMemoryFault(1_736_034_666, 1_000_000_000, 750_000_000, 1, 12, 2), e.getFault());
final MSQFault fault = Assert.assertThrows(MSQException.class, () -> create(1_000_000_000, 2, 32, 2, 1, 0, 0))
.getFault();
Assert.assertEquals(new NotEnoughMemoryFault(4_048_090_666L, 1_000_000_000, 750_000_000, 2, 32, 2), fault);
Assert.assertEquals(
new WorkerMemoryParameters(673_000_000, frameSize, 1, 604, 67_300_000, 200_000_000),
WorkerMemoryParameters.createInstance(memoryIntrospector, frameSize, slices, broadcastInputs, shuffleSpec, 1, 1)
);
}
@Test
public void test_oneWorkerInJvm_twoHundredWorkersInCluster()
public void test_1WorkerInJvm_alone_4Threads()
{
Assert.assertEquals(params(474_000_000, 1, 83, 150_000_000), create(2_000_000_000, 1, 1, 1, 200, 0, 0));
Assert.assertEquals(params(249_000_000, 2, 27, 150_000_000), create(2_000_000_000, 1, 2, 1, 200, 0, 0));
final int numThreads = 4;
final int frameSize = WorkerMemoryParameters.DEFAULT_FRAME_SIZE;
final MSQException e = Assert.assertThrows(
MSQException.class,
() -> create(1_000_000_000, 1, 4, 1, 200, 0, 0)
final MemoryIntrospectorImpl memoryIntrospector = createMemoryIntrospector(1_250_000_000, 1, numThreads);
final List<InputSlice> slices = makeInputSlices(ReadablePartitions.striped(0, 1, numThreads));
final IntSet broadcastInputs = IntSets.emptySet();
final ShuffleSpec shuffleSpec = makeSortShuffleSpec();
Assert.assertEquals(
new WorkerMemoryParameters(892_000_000, frameSize, 4, 199, 22_300_000, 0),
WorkerMemoryParameters.createInstance(memoryIntrospector, frameSize, slices, broadcastInputs, shuffleSpec, 1, 1)
);
Assert.assertEquals(new TooManyWorkersFault(200, 109), e.getFault());
}
@Test
public void test_fourWorkersInJvm_twoHundredWorkersInCluster()
public void test_1WorkerInJvm_alone_withBroadcast_4Threads()
{
Assert.assertEquals(params(1_014_000_000, 1, 150, 168_750_000), create(9_000_000_000L, 4, 1, 1, 200, 0, 0));
Assert.assertEquals(params(811_500_000, 2, 62, 168_750_000), create(9_000_000_000L, 4, 2, 1, 200, 0, 0));
Assert.assertEquals(params(558_375_000, 4, 22, 168_750_000), create(9_000_000_000L, 4, 4, 1, 200, 0, 0));
Assert.assertEquals(params(305_250_000, 4, 14, 168_750_000), create(9_000_000_000L, 4, 8, 1, 200, 0, 0));
Assert.assertEquals(params(102_750_000, 4, 8, 168_750_000), create(9_000_000_000L, 4, 16, 1, 200, 0, 0));
final int numThreads = 4;
final int frameSize = WorkerMemoryParameters.DEFAULT_FRAME_SIZE;
final MSQException e = Assert.assertThrows(
MSQException.class,
() -> create(8_000_000_000L, 4, 32, 1, 200, 0, 0)
final MemoryIntrospectorImpl memoryIntrospector = createMemoryIntrospector(1_250_000_000, 1, numThreads);
final List<InputSlice> slices = makeInputSlices(
ReadablePartitions.striped(0, 1, numThreads),
ReadablePartitions.striped(0, 1, 1)
);
final IntSet broadcastInputs = IntSets.singleton(1);
final ShuffleSpec shuffleSpec = makeSortShuffleSpec();
Assert.assertEquals(new TooManyWorkersFault(200, 124), e.getFault());
// Make sure 124 actually works, and 125 doesn't. (Verify the error message above.)
Assert.assertEquals(params(25_000_000, 4, 3, 150_000_000), create(8_000_000_000L, 4, 32, 1, 124, 0, 0));
final MSQException e2 = Assert.assertThrows(
MSQException.class,
() -> create(8_000_000_000L, 4, 32, 1, 125, 0, 0)
Assert.assertEquals(
new WorkerMemoryParameters(592_000_000, frameSize, 4, 132, 14_800_000, 200_000_000),
WorkerMemoryParameters.createInstance(memoryIntrospector, frameSize, slices, broadcastInputs, shuffleSpec, 1, 1)
);
Assert.assertEquals(new TooManyWorkersFault(125, 124), e2.getFault());
}
@Test
public void test_fourWorkersInJvm_twoHundredWorkersInCluster_twoConcurrentStages()
public void test_1WorkerInJvm_alone_noStats_4Threads()
{
Assert.assertEquals(params(406_500_000, 1, 74, 84_375_000), create(9_000_000_000L, 4, 1, 2, 200, 0, 0));
Assert.assertEquals(params(305_250_000, 2, 30, 84_375_000), create(9_000_000_000L, 4, 2, 2, 200, 0, 0));
Assert.assertEquals(params(178_687_500, 4, 10, 84_375_000), create(9_000_000_000L, 4, 4, 2, 200, 0, 0));
Assert.assertEquals(params(52_125_000, 4, 6, 84_375_000), create(9_000_000_000L, 4, 8, 2, 200, 0, 0));
final int numThreads = 4;
final int frameSize = WorkerMemoryParameters.DEFAULT_FRAME_SIZE;
final MSQException e = Assert.assertThrows(
MSQException.class,
() -> create(8_000_000_000L, 4, 16, 2, 200, 0, 0)
final MemoryIntrospectorImpl memoryIntrospector = createMemoryIntrospector(1_250_000_000, 1, 4);
final List<InputSlice> slices = makeInputSlices(ReadablePartitions.striped(0, 1, numThreads));
final IntSet broadcastInputs = IntSets.emptySet();
final ShuffleSpec shuffleSpec = null;
Assert.assertEquals(
new WorkerMemoryParameters(892_000_000, frameSize, 4, 222, 0, 0),
WorkerMemoryParameters.createInstance(memoryIntrospector, frameSize, slices, broadcastInputs, shuffleSpec, 1, 1)
);
Assert.assertEquals(new TooManyWorkersFault(200, 109), e.getFault());
// Make sure 109 actually works, and 110 doesn't. (Verify the error message above.)
Assert.assertEquals(params(25_000_000, 4, 3, 75_000_000), create(8_000_000_000L, 4, 16, 2, 109, 0, 0));
final MSQException e2 = Assert.assertThrows(
MSQException.class,
() -> create(8_000_000_000L, 4, 16, 2, 110, 0, 0)
);
Assert.assertEquals(new TooManyWorkersFault(110, 109), e2.getFault());
}
@Test
public void test_oneWorkerInJvm_smallWorkerCapacity()
public void test_1WorkerInJvm_alone_2ConcurrentStages_4Threads()
{
// Supersorter max channels per processer are one less than they are usually to account for extra frames that are required while creating composing output channels
Assert.assertEquals(params(41_200_000, 1, 3, 9_600_000), create(128_000_000, 1, 1, 1, 1, 0, 0));
Assert.assertEquals(params(26_800_000, 1, 1, 9_600_000), create(128_000_000, 1, 2, 1, 1, 0, 0));
final int numThreads = 4;
final int frameSize = WorkerMemoryParameters.DEFAULT_FRAME_SIZE;
final MSQException e = Assert.assertThrows(
MSQException.class,
() -> create(1_000_000_000, 1, 32, 1, 1, 0, 0)
final MemoryIntrospectorImpl memoryIntrospector = createMemoryIntrospector(1_250_000_000, 1, numThreads);
final List<InputSlice> slices = makeInputSlices(ReadablePartitions.striped(0, 1, numThreads));
final IntSet broadcastInputs = IntSets.emptySet();
final ShuffleSpec shuffleSpec = makeSortShuffleSpec();
Assert.assertEquals(
new WorkerMemoryParameters(392_000_000, frameSize, 4, 87, 9_800_000, 0),
WorkerMemoryParameters.createInstance(memoryIntrospector, frameSize, slices, broadcastInputs, shuffleSpec, 2, 1)
);
Assert.assertEquals(new NotEnoughMemoryFault(1_588_044_000, 1_000_000_000, 750_000_000, 1, 32, 1), e.getFault());
final MSQException e2 = Assert.assertThrows(
MSQException.class,
() -> create(128_000_000, 1, 4, 1, 1, 0, 0)
);
Assert.assertEquals(new NotEnoughMemoryFault(580_006_666, 12_8000_000, 96_000_000, 1, 4, 1), e2.getFault());
final MSQFault fault = Assert.assertThrows(MSQException.class, () -> create(1_000_000_000, 2, 32, 1, 1, 0, 0))
.getFault();
Assert.assertEquals(new NotEnoughMemoryFault(2024045333, 1_000_000_000, 750_000_000, 2, 32, 1), fault);
}
@Test
public void test_fourWorkersInJvm_twoHundredWorkersInCluster_hashPartitions()
public void test_1WorkerInJvm_alone_2ConcurrentStages_4Threads_highHeap()
{
Assert.assertEquals(params(814_000_000, 1, 150, 168_750_000), create(9_000_000_000L, 4, 1, 1, 200, 200, 0));
Assert.assertEquals(params(611_500_000, 2, 62, 168_750_000), create(9_000_000_000L, 4, 2, 1, 200, 200, 0));
Assert.assertEquals(params(358_375_000, 4, 22, 168_750_000), create(9_000_000_000L, 4, 4, 1, 200, 200, 0));
Assert.assertEquals(params(105_250_000, 4, 14, 168_750_000), create(9_000_000_000L, 4, 8, 1, 200, 200, 0));
final int numThreads = 4;
final int frameSize = WorkerMemoryParameters.DEFAULT_FRAME_SIZE;
final MSQException e = Assert.assertThrows(
MSQException.class,
() -> create(9_000_000_000L, 4, 16, 1, 200, 200, 0)
final MemoryIntrospectorImpl memoryIntrospector = createMemoryIntrospector(6_250_000_000L, 1, numThreads);
final List<InputSlice> slices = makeInputSlices(ReadablePartitions.striped(0, 1, numThreads));
final IntSet broadcastInputs = IntSets.emptySet();
final ShuffleSpec shuffleSpec = makeSortShuffleSpec();
Assert.assertEquals(
new WorkerMemoryParameters(2_392_000_000L, frameSize, 4, 537, 59_800_000, 0),
WorkerMemoryParameters.createInstance(memoryIntrospector, frameSize, slices, broadcastInputs, shuffleSpec, 2, 1)
);
Assert.assertEquals(new TooManyWorkersFault(200, 138), e.getFault());
// Make sure 138 actually works, and 139 doesn't. (Verify the error message above.)
Assert.assertEquals(params(26_750_000, 4, 8, 168_750_000), create(9_000_000_000L, 4, 16, 1, 138, 138, 0));
final MSQException e2 = Assert.assertThrows(
MSQException.class,
() -> create(9_000_000_000L, 4, 16, 1, 139, 139, 0)
);
Assert.assertEquals(new TooManyWorkersFault(139, 138), e2.getFault());
}
@Test
public void test_oneWorkerInJvm_oneByteUsableMemory()
public void test_1WorkerInJvm_alone_32Threads()
{
final int numThreads = 32;
final int frameSize = WorkerMemoryParameters.DEFAULT_FRAME_SIZE;
final MemoryIntrospectorImpl memoryIntrospector = createMemoryIntrospector(1_250_000_000, 1, numThreads);
final List<InputSlice> slices = makeInputSlices(ReadablePartitions.striped(0, 1, numThreads));
final IntSet broadcastInputs = IntSets.emptySet();
final ShuffleSpec shuffleSpec = makeSortShuffleSpec();
Assert.assertEquals(
new WorkerMemoryParameters(136_000_000, frameSize, 32, 2, 425_000, 0),
WorkerMemoryParameters.createInstance(memoryIntrospector, frameSize, slices, broadcastInputs, shuffleSpec, 1, 1)
);
}
@Test
public void test_1WorkerInJvm_alone_33Threads()
{
final int numThreads = 33;
final int frameSize = WorkerMemoryParameters.DEFAULT_FRAME_SIZE;
final MemoryIntrospectorImpl memoryIntrospector = createMemoryIntrospector(1_250_000_000, 1, numThreads);
final List<InputSlice> slices = makeInputSlices(ReadablePartitions.striped(0, 1, numThreads));
final IntSet broadcastInputs = IntSets.emptySet();
final ShuffleSpec shuffleSpec = makeSortShuffleSpec();
Assert.assertEquals(
new WorkerMemoryParameters(109_000_000, frameSize, 32, 2, 330_303, 0),
WorkerMemoryParameters.createInstance(memoryIntrospector, frameSize, slices, broadcastInputs, shuffleSpec, 1, 1)
);
}
@Test
public void test_1WorkerInJvm_alone_40Threads()
{
final int numThreads = 40;
final int frameSize = WorkerMemoryParameters.DEFAULT_FRAME_SIZE;
final MemoryIntrospectorImpl memoryIntrospector = createMemoryIntrospector(1_250_000_000, 1, numThreads);
final List<InputSlice> slices = makeInputSlices(ReadablePartitions.striped(0, 1, numThreads));
final IntSet broadcastInputs = IntSets.emptySet();
final ShuffleSpec shuffleSpec = makeSortShuffleSpec();
final MSQException e = Assert.assertThrows(
MSQException.class,
() -> WorkerMemoryParameters.createInstance(1, 1, 1, 1, 32, 1, 1)
() -> WorkerMemoryParameters.createInstance(
memoryIntrospector,
frameSize,
slices,
broadcastInputs,
shuffleSpec,
1,
1
)
);
Assert.assertEquals(new NotEnoughMemoryFault(554669334, 1, 1, 1, 1, 1), e.getFault());
Assert.assertEquals(
new NotEnoughMemoryFault(1_366_250_000, 1_250_000_000, 1_000_000_000, 1, 40, 1, 1),
e.getFault()
);
}
@Test
public void test_1WorkerInJvm_alone_40Threads_slightlyLessMemoryThanError()
{
// Test with one byte less than the amount of memory recommended in the error message
// for test_1WorkerInJvm_alone_40Threads.
final int numThreads = 40;
final int frameSize = WorkerMemoryParameters.DEFAULT_FRAME_SIZE;
final MemoryIntrospectorImpl memoryIntrospector = createMemoryIntrospector(1_366_250_000 - 1, 1, numThreads);
final List<InputSlice> slices = makeInputSlices(ReadablePartitions.striped(0, 1, numThreads));
final IntSet broadcastInputs = IntSets.emptySet();
final ShuffleSpec shuffleSpec = makeSortShuffleSpec();
final MSQException e = Assert.assertThrows(
MSQException.class,
() -> WorkerMemoryParameters.createInstance(
memoryIntrospector,
frameSize,
slices,
broadcastInputs,
shuffleSpec,
1,
1
)
);
Assert.assertEquals(
new NotEnoughMemoryFault(1_366_250_000, 1_366_249_999, 1_092_999_999, 1, 40, 1, 1),
e.getFault()
);
}
@Test
public void test_1WorkerInJvm_alone_40Threads_memoryFromError()
{
// Test with the amount of memory recommended in the error message for test_1WorkerInJvm_alone_40Threads.
final int numThreads = 40;
final int frameSize = WorkerMemoryParameters.DEFAULT_FRAME_SIZE;
final MemoryIntrospectorImpl memoryIntrospector = createMemoryIntrospector(1_366_250_000, 1, numThreads);
final List<InputSlice> slices = makeInputSlices(ReadablePartitions.striped(0, 1, numThreads));
final IntSet broadcastInputs = IntSets.emptySet();
final ShuffleSpec shuffleSpec = makeSortShuffleSpec();
Assert.assertEquals(
new WorkerMemoryParameters(13_000_000, frameSize, 1, 2, 250_000, 0),
WorkerMemoryParameters.createInstance(memoryIntrospector, frameSize, slices, broadcastInputs, shuffleSpec, 1, 1)
);
}
@Test
public void test_1WorkerInJvm_200WorkersInCluster_4Threads()
{
final int numThreads = 4;
final int frameSize = WorkerMemoryParameters.DEFAULT_FRAME_SIZE;
final MemoryIntrospectorImpl memoryIntrospector = createMemoryIntrospector(2_500_000_000L, 1, numThreads);
final List<InputSlice> slices = makeInputSlices(ReadablePartitions.striped(0, 200, numThreads));
final IntSet broadcastInputs = IntSets.emptySet();
final ShuffleSpec shuffleSpec = makeSortShuffleSpec();
Assert.assertEquals(
new WorkerMemoryParameters(1_096_000_000, frameSize, 4, 245, 27_400_000, 0),
WorkerMemoryParameters.createInstance(memoryIntrospector, frameSize, slices, broadcastInputs, shuffleSpec, 1, 1)
);
}
@Test
public void test_1WorkerInJvm_200WorkersInCluster_4Threads_2OutputPartitions()
{
final int numThreads = 4;
final int frameSize = WorkerMemoryParameters.DEFAULT_FRAME_SIZE;
final MemoryIntrospectorImpl memoryIntrospector = createMemoryIntrospector(2_500_000_000L, 1, numThreads);
final List<InputSlice> slices = makeInputSlices(ReadablePartitions.striped(0, 200, 2));
final IntSet broadcastInputs = IntSets.emptySet();
final ShuffleSpec shuffleSpec = makeSortShuffleSpec();
Assert.assertEquals(
new WorkerMemoryParameters(1_548_000_000, frameSize, 4, 347, 38_700_000, 0),
WorkerMemoryParameters.createInstance(memoryIntrospector, frameSize, slices, broadcastInputs, shuffleSpec, 1, 1)
);
}
@Test
public void test_1WorkerInJvm_200WorkersInCluster_2ConcurrentStages_4Threads()
{
final int numThreads = 4;
final int frameSize = WorkerMemoryParameters.DEFAULT_FRAME_SIZE;
final MemoryIntrospectorImpl memoryIntrospector = createMemoryIntrospector(2_500_000_000L, 1, numThreads);
final List<InputSlice> slices = makeInputSlices(ReadablePartitions.striped(0, 200, numThreads));
final IntSet broadcastInputs = IntSets.emptySet();
final ShuffleSpec shuffleSpec = makeSortShuffleSpec();
Assert.assertEquals(
new WorkerMemoryParameters(96_000_000, frameSize, 4, 20, 2_500_000, 0),
WorkerMemoryParameters.createInstance(memoryIntrospector, frameSize, slices, broadcastInputs, shuffleSpec, 2, 1)
);
}
@Test
public void test_12WorkersInJvm_200WorkersInCluster_64Threads_4OutputPartitions()
{
final int numThreads = 64;
final int frameSize = WorkerMemoryParameters.DEFAULT_FRAME_SIZE;
final MemoryIntrospectorImpl memoryIntrospector = createMemoryIntrospector(40_000_000_000L, 12, numThreads);
final List<InputSlice> slices = makeInputSlices(ReadablePartitions.striped(0, 200, 4));
final IntSet broadcastInputs = IntSets.emptySet();
final ShuffleSpec shuffleSpec = makeSortShuffleSpec();
Assert.assertEquals(
new WorkerMemoryParameters(1_762_666_666, frameSize, 64, 23, 2_754_166, 0),
WorkerMemoryParameters.createInstance(memoryIntrospector, frameSize, slices, broadcastInputs, shuffleSpec, 1, 1)
);
}
@Test
public void test_12WorkersInJvm_200WorkersInCluster_2ConcurrentStages_64Threads_4OutputPartitions()
{
final int numThreads = 64;
final int frameSize = WorkerMemoryParameters.DEFAULT_FRAME_SIZE;
final MemoryIntrospectorImpl memoryIntrospector = createMemoryIntrospector(40_000_000_000L, 12, numThreads);
final List<InputSlice> slices = makeInputSlices(ReadablePartitions.striped(0, 200, 4));
final IntSet broadcastInputs = IntSets.emptySet();
final ShuffleSpec shuffleSpec = makeSortShuffleSpec();
Assert.assertEquals(
new WorkerMemoryParameters(429_333_333, frameSize, 64, 5, 670_833, 0),
WorkerMemoryParameters.createInstance(memoryIntrospector, frameSize, slices, broadcastInputs, shuffleSpec, 2, 1)
);
}
@Test
public void test_1WorkerInJvm_MaxWorkersInCluster_2ConcurrentStages_2Threads()
{
final int numWorkers = Limits.MAX_WORKERS;
final int numThreads = 2;
final int frameSize = WorkerMemoryParameters.DEFAULT_FRAME_SIZE;
final MemoryIntrospectorImpl memoryIntrospector = createMemoryIntrospector(6_250_000_000L, 1, numThreads);
final List<InputSlice> slices = makeInputSlices(ReadablePartitions.striped(0, numWorkers, numThreads));
final IntSet broadcastInputs = IntSets.emptySet();
final ShuffleSpec shuffleSpec = makeSortShuffleSpec();
Assert.assertEquals(
new WorkerMemoryParameters(448_000_000, frameSize, 2, 200, 22_400_000, 0),
WorkerMemoryParameters.createInstance(memoryIntrospector, frameSize, slices, broadcastInputs, shuffleSpec, 2, 1)
);
}
@Test
public void test_1WorkerInJvm_MaxWorkersInCluster_1Thread()
{
final int numWorkers = Limits.MAX_WORKERS;
final int numThreads = 1;
final int frameSize = WorkerMemoryParameters.DEFAULT_FRAME_SIZE;
final MemoryIntrospectorImpl memoryIntrospector = createMemoryIntrospector(2_500_000_000L, 1, numThreads);
final List<InputSlice> slices = makeInputSlices(ReadablePartitions.striped(0, numWorkers, numThreads));
final IntSet broadcastInputs = IntSets.emptySet();
final ShuffleSpec shuffleSpec = makeSortShuffleSpec();
Assert.assertEquals(
new WorkerMemoryParameters(974_000_000, frameSize, 1, 875, 97_400_000, 0),
WorkerMemoryParameters.createInstance(memoryIntrospector, frameSize, slices, broadcastInputs, shuffleSpec, 1, 1)
);
}
@Test
@ -206,39 +408,28 @@ public class WorkerMemoryParametersTest
EqualsVerifier.forClass(WorkerMemoryParameters.class).usingGetClass().verify();
}
private static WorkerMemoryParameters params(
final long processorBundleMemory,
final int superSorterMaxActiveProcessors,
final int superSorterMaxChannelsPerProcessor,
final int partitionStatisticsMaxRetainedBytes
private static MemoryIntrospectorImpl createMemoryIntrospector(
final long totalMemory,
final int numTasksInJvm,
final int numProcessingThreads
)
{
return new WorkerMemoryParameters(
processorBundleMemory,
superSorterMaxActiveProcessors,
superSorterMaxChannelsPerProcessor,
partitionStatisticsMaxRetainedBytes
);
return new MemoryIntrospectorImpl(totalMemory, 0.8, numTasksInJvm, numProcessingThreads, null);
}
private static WorkerMemoryParameters create(
final long maxMemoryInJvm,
final int numWorkersInJvm,
final int numProcessingThreadsInJvm,
final int maxConcurrentStages,
final int numInputWorkers,
final int numHashOutputPartitions,
final int totalLookUpFootprint
)
private static List<InputSlice> makeInputSlices(final ReadablePartitions... partitionss)
{
return WorkerMemoryParameters.createInstance(
maxMemoryInJvm,
numWorkersInJvm,
numProcessingThreadsInJvm,
maxConcurrentStages,
numInputWorkers,
numHashOutputPartitions,
totalLookUpFootprint
return Arrays.stream(partitionss)
.map(partitions -> new StageInputSlice(0, partitions, OutputChannelMode.LOCAL_STORAGE))
.collect(Collectors.toList());
}
private static ShuffleSpec makeSortShuffleSpec()
{
return new GlobalSortTargetSizeShuffleSpec(
new ClusterBy(ImmutableList.of(new KeyColumn("foo", KeyOrder.ASCENDING)), 0),
1_000_000,
false
);
}
}

View File

@ -59,6 +59,7 @@ public class IndexerWorkerContextTest
null,
null,
null,
null,
null
);
}

View File

@ -74,7 +74,7 @@ public class MSQFaultSerdeTest
));
assertFaultSerde(new InvalidNullByteFault("the source", 1, "the column", "the value", 2));
assertFaultSerde(new InvalidFieldFault("the source", "the column", 1, "the error", "the log msg"));
assertFaultSerde(new NotEnoughMemoryFault(1000, 1000, 900, 1, 2, 2));
assertFaultSerde(new NotEnoughMemoryFault(1234, 1000, 1000, 900, 1, 2, 2));
assertFaultSerde(QueryNotSupportedFault.INSTANCE);
assertFaultSerde(new QueryRuntimeFault("new error", "base error"));
assertFaultSerde(new QueryRuntimeFault("new error", null));

View File

@ -32,9 +32,9 @@ import org.apache.druid.frame.channel.WritableFrameChannel;
import org.apache.druid.frame.processor.Bouncer;
import org.apache.druid.frame.processor.FrameProcessorExecutorTest;
import org.apache.druid.frame.processor.FrameProcessors;
import org.apache.druid.frame.processor.manager.NilFrameProcessor;
import org.apache.druid.frame.processor.manager.ProcessorManager;
import org.apache.druid.frame.processor.manager.ProcessorManagers;
import org.apache.druid.frame.processor.manager.SequenceProcessorManagerTest;
import org.apache.druid.frame.processor.test.SimpleReturningFrameProcessor;
import org.apache.druid.frame.processor.test.SingleChannelFrameProcessor;
import org.apache.druid.frame.processor.test.SingleRowWritingFrameProcessor;
@ -184,7 +184,7 @@ public class ChainedProcessorManagerTest extends FrameProcessorExecutorTest.Base
ProcessorManagers.of(
ImmutableList.of(
new SimpleReturningFrameProcessor<>(ImmutableList.of(4L, 5L, 6L)),
new SequenceProcessorManagerTest.NilFrameProcessor<>()
new NilFrameProcessor<>()
)
),
(values) -> createNextProcessors(

View File

@ -428,6 +428,18 @@ public class MSQTestBase extends BaseCalciteQueryTest
binder -> {
DruidProcessingConfig druidProcessingConfig = new DruidProcessingConfig()
{
@Override
public int getNumThreads()
{
return 1;
}
@Override
public int intermediateComputeSizeBytes()
{
return 10_000_000;
}
@Override
public String getFormatString()
{
@ -750,14 +762,13 @@ public class MSQTestBase extends BaseCalciteQueryTest
public static WorkerMemoryParameters makeTestWorkerMemoryParameters()
{
return WorkerMemoryParameters.createInstance(
WorkerMemoryParameters.PROCESSING_MINIMUM_BYTES * 50,
2,
10,
return new WorkerMemoryParameters(
100_000_000,
WorkerMemoryParameters.DEFAULT_FRAME_SIZE,
1,
2,
1,
0
50,
10_000_000,
10_000_000
);
}

View File

@ -21,20 +21,21 @@ package org.apache.druid.msq.test;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.inject.Injector;
import org.apache.druid.collections.StupidPool;
import org.apache.druid.frame.processor.Bouncer;
import org.apache.druid.java.util.common.FileUtils;
import org.apache.druid.java.util.common.io.Closer;
import org.apache.druid.msq.exec.Controller;
import org.apache.druid.msq.exec.ControllerClient;
import org.apache.druid.msq.exec.DataServerQueryHandlerFactory;
import org.apache.druid.msq.exec.OutputChannelMode;
import org.apache.druid.msq.exec.ProcessingBuffers;
import org.apache.druid.msq.exec.Worker;
import org.apache.druid.msq.exec.WorkerClient;
import org.apache.druid.msq.exec.WorkerContext;
import org.apache.druid.msq.exec.WorkerMemoryParameters;
import org.apache.druid.msq.exec.WorkerStorageParameters;
import org.apache.druid.msq.kernel.FrameContext;
import org.apache.druid.msq.kernel.QueryDefinition;
import org.apache.druid.msq.kernel.WorkOrder;
import org.apache.druid.msq.querykit.DataSegmentProvider;
import org.apache.druid.query.groupby.GroupingEngine;
import org.apache.druid.segment.IndexIO;
@ -48,6 +49,7 @@ import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFacto
import org.apache.druid.server.DruidNode;
import java.io.File;
import java.nio.ByteBuffer;
import java.util.Map;
public class MSQTestWorkerContext implements WorkerContext
@ -58,7 +60,6 @@ public class MSQTestWorkerContext implements WorkerContext
private final Injector injector;
private final Map<String, Worker> inMemoryWorkers;
private final File file = FileUtils.createTempDir();
private final Bouncer bouncer = new Bouncer(1);
private final WorkerMemoryParameters workerMemoryParameters;
private final WorkerStorageParameters workerStorageParameters;
@ -130,9 +131,9 @@ public class MSQTestWorkerContext implements WorkerContext
}
@Override
public FrameContext frameContext(QueryDefinition queryDef, int stageNumber, OutputChannelMode outputChannelMode)
public FrameContext frameContext(WorkOrder workOrder)
{
return new FrameContextImpl(new File(tempDir(), queryDef.getStageDefinition(stageNumber).getId().toString()));
return new FrameContextImpl(new File(tempDir(), workOrder.getStageDefinition().getId().toString()));
}
@Override
@ -240,9 +241,12 @@ public class MSQTestWorkerContext implements WorkerContext
}
@Override
public Bouncer processorBouncer()
public ProcessingBuffers processingBuffers()
{
return bouncer;
return new ProcessingBuffers(
new StupidPool<>("testProcessing", () -> ByteBuffer.allocate(1_000_000)),
new Bouncer(1)
);
}
@Override

View File

@ -0,0 +1,48 @@
/*
* 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.collections;
import java.util.NoSuchElementException;
import java.util.concurrent.BlockingQueue;
/**
* Implementation of {@link NonBlockingPool} based on a pre-created {@link BlockingQueue} that never actually blocks.
* If the pool is empty when {@link #take()} is called, it throws {@link NoSuchElementException}.
*/
public class QueueNonBlockingPool<T> implements NonBlockingPool<T>
{
private final BlockingQueue<T> queue;
public QueueNonBlockingPool(final BlockingQueue<T> queue)
{
this.queue = queue;
}
@Override
public ResourceHolder<T> take()
{
final T item = queue.poll();
if (item == null) {
throw new NoSuchElementException("No items available");
}
return new ReferenceCountingResourceHolder<>(item, () -> queue.add(item));
}
}

View File

@ -0,0 +1,74 @@
/*
* 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.manager;
import com.google.common.util.concurrent.ListenableFuture;
import org.apache.druid.common.guava.FutureUtils;
import org.apache.druid.frame.processor.Bouncer;
import java.util.Optional;
/**
* Manager that limits the number of processors that may run concurrently.
*/
public class ConcurrencyLimitedProcessorManager<T, R> implements ProcessorManager<T, R>
{
private final ProcessorManager<T, R> delegate;
private final Bouncer bouncer;
public ConcurrencyLimitedProcessorManager(ProcessorManager<T, R> delegate, int limit)
{
this.delegate = delegate;
this.bouncer = new Bouncer(limit);
}
@Override
public ListenableFuture<Optional<ProcessorAndCallback<T>>> next()
{
final ListenableFuture<Bouncer.Ticket> ticket = bouncer.ticket();
return FutureUtils.transformAsync(
ticket,
t -> FutureUtils.transform(
delegate.next(),
nextProcessor -> nextProcessor.map(
retVal -> new ProcessorAndCallback<>(
retVal.processor(),
r -> {
FutureUtils.getUncheckedImmediately(ticket).giveBack();
retVal.onComplete(r);
}
)
)
)
);
}
@Override
public R result()
{
return delegate.result();
}
@Override
public void close()
{
delegate.close();
}
}

View File

@ -21,6 +21,8 @@ package org.apache.druid.query.groupby;
import com.google.common.annotations.VisibleForTesting;
import com.google.inject.Inject;
import org.apache.druid.collections.NonBlockingPool;
import org.apache.druid.guice.annotations.Global;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.guava.Sequence;
import org.apache.druid.query.Query;
@ -36,6 +38,8 @@ import org.apache.druid.segment.TimeBoundaryInspector;
import javax.annotation.Nullable;
import java.nio.ByteBuffer;
/**
*
*/
@ -43,21 +47,24 @@ public class GroupByQueryRunnerFactory implements QueryRunnerFactory<ResultRow,
{
private final GroupingEngine groupingEngine;
private final GroupByQueryQueryToolChest toolChest;
private final NonBlockingPool<ByteBuffer> processingBufferPool;
@Inject
public GroupByQueryRunnerFactory(
GroupingEngine groupingEngine,
GroupByQueryQueryToolChest toolChest
GroupByQueryQueryToolChest toolChest,
@Global NonBlockingPool<ByteBuffer> processingBufferPool
)
{
this.groupingEngine = groupingEngine;
this.toolChest = toolChest;
this.processingBufferPool = processingBufferPool;
}
@Override
public QueryRunner<ResultRow> createRunner(final Segment segment)
{
return new GroupByQueryRunner(segment, groupingEngine);
return new GroupByQueryRunner(segment, groupingEngine, processingBufferPool);
}
/**
@ -69,14 +76,9 @@ public class GroupByQueryRunnerFactory implements QueryRunnerFactory<ResultRow,
final Iterable<QueryRunner<ResultRow>> queryRunners
)
{
return new QueryRunner<ResultRow>()
{
@Override
public Sequence<ResultRow> run(QueryPlus<ResultRow> queryPlus, ResponseContext responseContext)
{
QueryRunner<ResultRow> rowQueryRunner = groupingEngine.mergeRunners(queryProcessingPool, queryRunners);
return rowQueryRunner.run(queryPlus, responseContext);
}
return (queryPlus, responseContext) -> {
QueryRunner<ResultRow> rowQueryRunner = groupingEngine.mergeRunners(queryProcessingPool, queryRunners);
return rowQueryRunner.run(queryPlus, responseContext);
};
}
@ -92,12 +94,18 @@ public class GroupByQueryRunnerFactory implements QueryRunnerFactory<ResultRow,
@Nullable
private final TimeBoundaryInspector timeBoundaryInspector;
private final GroupingEngine groupingEngine;
private final NonBlockingPool<ByteBuffer> processingBufferPool;
public GroupByQueryRunner(Segment segment, final GroupingEngine groupingEngine)
public GroupByQueryRunner(
Segment segment,
final GroupingEngine groupingEngine,
final NonBlockingPool<ByteBuffer> processingBufferPool
)
{
this.cursorFactory = segment.asCursorFactory();
this.timeBoundaryInspector = segment.as(TimeBoundaryInspector.class);
this.groupingEngine = groupingEngine;
this.processingBufferPool = processingBufferPool;
}
@Override
@ -112,6 +120,7 @@ public class GroupByQueryRunnerFactory implements QueryRunnerFactory<ResultRow,
(GroupByQuery) query,
cursorFactory,
timeBoundaryInspector,
processingBufferPool,
(GroupByQueryMetrics) queryPlus.getQueryMetrics()
);
}

View File

@ -32,7 +32,6 @@ import org.apache.druid.collections.NonBlockingPool;
import org.apache.druid.collections.ReferenceCountingResourceHolder;
import org.apache.druid.collections.ResourceHolder;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.guice.annotations.Global;
import org.apache.druid.guice.annotations.Json;
import org.apache.druid.guice.annotations.Merging;
import org.apache.druid.guice.annotations.Smile;
@ -118,8 +117,7 @@ public class GroupingEngine
private final DruidProcessingConfig processingConfig;
private final Supplier<GroupByQueryConfig> configSupplier;
private final NonBlockingPool<ByteBuffer> bufferPool;
GroupByResourcesReservationPool groupByResourcesReservationPool;
private final GroupByResourcesReservationPool groupByResourcesReservationPool;
private final ObjectMapper jsonMapper;
private final ObjectMapper spillMapper;
private final QueryWatcher queryWatcher;
@ -128,7 +126,6 @@ public class GroupingEngine
public GroupingEngine(
DruidProcessingConfig processingConfig,
Supplier<GroupByQueryConfig> configSupplier,
@Global NonBlockingPool<ByteBuffer> bufferPool,
@Merging GroupByResourcesReservationPool groupByResourcesReservationPool,
@Json ObjectMapper jsonMapper,
@Smile ObjectMapper spillMapper,
@ -137,7 +134,6 @@ public class GroupingEngine
{
this.processingConfig = processingConfig;
this.configSupplier = configSupplier;
this.bufferPool = bufferPool;
this.groupByResourcesReservationPool = groupByResourcesReservationPool;
this.jsonMapper = jsonMapper;
this.spillMapper = spillMapper;
@ -470,6 +466,8 @@ public class GroupingEngine
* @param query the groupBy query
* @param cursorFactory cursor factory for the segment in question
* @param timeBoundaryInspector time boundary inspector for the segment in question
* @param bufferPool processing buffer pool
* @param groupByQueryMetrics metrics instance, will be populated if nonnull
*
* @return result sequence for the cursor factory
*/
@ -477,6 +475,7 @@ public class GroupingEngine
GroupByQuery query,
CursorFactory cursorFactory,
@Nullable TimeBoundaryInspector timeBoundaryInspector,
NonBlockingPool<ByteBuffer> bufferPool,
@Nullable GroupByQueryMetrics groupByQueryMetrics
)
{

View File

@ -0,0 +1,76 @@
/*
* 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.collections;
import org.junit.Assert;
import org.junit.Test;
import java.util.NoSuchElementException;
import java.util.concurrent.ArrayBlockingQueue;
import java.util.concurrent.BlockingQueue;
public class QueueNonBlockingPoolTest
{
@Test
public void testTakeAllTwice()
{
final BlockingQueue<String> queue = new ArrayBlockingQueue<>(2);
queue.add("foo");
queue.add("bar");
final QueueNonBlockingPool<String> pool = new QueueNonBlockingPool<>(queue);
// Take everything from pool
final ResourceHolder<String> obj1 = pool.take();
Assert.assertEquals("foo", obj1.get());
Assert.assertEquals(1, queue.size());
final ResourceHolder<String> obj2 = pool.take();
Assert.assertEquals("bar", obj2.get());
Assert.assertEquals(0, queue.size());
Assert.assertThrows(
NoSuchElementException.class,
pool::take
);
// Re-fill pool in reverse order
obj2.close();
Assert.assertEquals(1, queue.size());
obj1.close();
Assert.assertEquals(2, queue.size());
// Re-take everything from pool
final ResourceHolder<String> obj1b = pool.take();
Assert.assertEquals("bar", obj1b.get());
Assert.assertEquals(1, queue.size());
final ResourceHolder<String> obj2b = pool.take();
Assert.assertEquals("foo", obj2b.get());
Assert.assertEquals(0, queue.size());
Assert.assertThrows(
NoSuchElementException.class,
pool::take
);
}
}

View File

@ -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.processor.manager;
import com.google.common.collect.ImmutableList;
import com.google.common.util.concurrent.ListenableFuture;
import org.apache.druid.java.util.common.Unit;
import org.junit.Assert;
import org.junit.Test;
import java.util.Optional;
public class ConcurrencyLimitedProcessorManagerTest
{
@Test
public void test_empty() throws Exception
{
try (final ConcurrencyLimitedProcessorManager<Object, Long> manager =
new ConcurrencyLimitedProcessorManager<>(ProcessorManagers.none(), 1)) {
final ListenableFuture<Optional<ProcessorAndCallback<Object>>> future = manager.next();
Assert.assertTrue(future.isDone());
Assert.assertFalse(future.get().isPresent());
Assert.assertEquals(0, (long) manager.result());
}
}
@Test
public void test_one_limitOne() throws Exception
{
final NilFrameProcessor<Unit> processor = new NilFrameProcessor<>();
try (final ConcurrencyLimitedProcessorManager<Unit, Long> manager =
new ConcurrencyLimitedProcessorManager<>(ProcessorManagers.of(ImmutableList.of(processor)), 1)) {
// First element.
ListenableFuture<Optional<ProcessorAndCallback<Unit>>> future = manager.next();
Assert.assertTrue(future.isDone());
Assert.assertTrue(future.get().isPresent());
Assert.assertSame(processor, future.get().get().processor());
// Simulate processor finishing.
future.get().get().onComplete(Unit.instance());
// End of sequence.
future = manager.next();
Assert.assertTrue(future.isDone());
Assert.assertFalse(future.get().isPresent());
}
}
@Test
public void test_two_limitOne() throws Exception
{
final NilFrameProcessor<Unit> processor0 = new NilFrameProcessor<>();
final NilFrameProcessor<Unit> processor1 = new NilFrameProcessor<>();
final ImmutableList<NilFrameProcessor<Unit>> processors = ImmutableList.of(processor0, processor1);
try (final ConcurrencyLimitedProcessorManager<Unit, Long> manager =
new ConcurrencyLimitedProcessorManager<>(ProcessorManagers.of(processors), 1)) {
// First element.
ListenableFuture<Optional<ProcessorAndCallback<Unit>>> future0 = manager.next();
Assert.assertTrue(future0.isDone());
Assert.assertTrue(future0.get().isPresent());
Assert.assertSame(processors.get(0), future0.get().get().processor());
// Second element. Not yet ready to run due to the limit.
ListenableFuture<Optional<ProcessorAndCallback<Unit>>> future1 = manager.next();
Assert.assertFalse(future1.isDone());
// Simulate processor0 finishing.
future0.get().get().onComplete(Unit.instance());
// processor1 is now ready to run.
Assert.assertTrue(future1.isDone());
Assert.assertTrue(future1.get().isPresent());
Assert.assertSame(processors.get(1), future1.get().get().processor());
// Simulate processor1 finishing.
future1.get().get().onComplete(Unit.instance());
// End of sequence.
future1 = manager.next();
Assert.assertTrue(future1.isDone());
Assert.assertFalse(future1.get().isPresent());
}
}
}

View File

@ -0,0 +1,60 @@
/*
* 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.manager;
import it.unimi.dsi.fastutil.ints.IntSet;
import org.apache.druid.frame.channel.ReadableFrameChannel;
import org.apache.druid.frame.channel.WritableFrameChannel;
import org.apache.druid.frame.processor.FrameProcessor;
import org.apache.druid.frame.processor.ReturnOrAwait;
import java.util.Collections;
import java.util.List;
/**
* Frame processor that throws an exception from {@link #runIncrementally(IntSet)}. Used as a dummy processor
* by tests of {@link ProcessorManager}.
*/
public class NilFrameProcessor<T> implements FrameProcessor<T>
{
@Override
public List<ReadableFrameChannel> inputChannels()
{
return Collections.emptyList();
}
@Override
public List<WritableFrameChannel> outputChannels()
{
return Collections.emptyList();
}
@Override
public ReturnOrAwait<T> runIncrementally(IntSet readableInputs)
{
throw new UnsupportedOperationException();
}
@Override
public void cleanup()
{
// Do nothing.
}
}

View File

@ -21,18 +21,13 @@ package org.apache.druid.frame.processor.manager;
import com.google.common.collect.ImmutableList;
import com.google.common.util.concurrent.ListenableFuture;
import it.unimi.dsi.fastutil.ints.IntSet;
import org.apache.druid.frame.channel.ReadableFrameChannel;
import org.apache.druid.frame.channel.WritableFrameChannel;
import org.apache.druid.frame.processor.FrameProcessor;
import org.apache.druid.frame.processor.ReturnOrAwait;
import org.apache.druid.java.util.common.Unit;
import org.apache.druid.java.util.common.guava.Sequences;
import org.junit.Assert;
import org.junit.Test;
import java.util.Collections;
import java.util.List;
import java.util.NoSuchElementException;
import java.util.Optional;
import java.util.concurrent.atomic.AtomicLong;
@ -59,7 +54,7 @@ public class SequenceProcessorManagerTest
@Test
public void test_one() throws Exception
{
final NilFrameProcessor processor = new NilFrameProcessor();
final NilFrameProcessor<Unit> processor = new NilFrameProcessor<>();
final AtomicLong closed = new AtomicLong();
try (final SequenceProcessorManager<Unit, FrameProcessor<Unit>> manager =
@ -84,8 +79,8 @@ public class SequenceProcessorManagerTest
@Test
public void test_two() throws Exception
{
final NilFrameProcessor processor0 = new NilFrameProcessor();
final NilFrameProcessor processor1 = new NilFrameProcessor();
final NilFrameProcessor<Unit> processor0 = new NilFrameProcessor<>();
final NilFrameProcessor<Unit> processor1 = new NilFrameProcessor<>();
final AtomicLong closed = new AtomicLong();
try (final SequenceProcessorManager<Unit, FrameProcessor<Unit>> manager =
@ -139,31 +134,4 @@ public class SequenceProcessorManagerTest
// Sequence is not closed because it never started iterating.
Assert.assertEquals(0, closed.get());
}
public static class NilFrameProcessor<T> implements FrameProcessor<T>
{
@Override
public List<ReadableFrameChannel> inputChannels()
{
return Collections.emptyList();
}
@Override
public List<WritableFrameChannel> outputChannels()
{
return Collections.emptyList();
}
@Override
public ReturnOrAwait<T> runIncrementally(IntSet readableInputs)
{
throw new UnsupportedOperationException();
}
@Override
public void cleanup()
{
// Do nothing.
}
}
}

View File

@ -336,7 +336,6 @@ public class GroupByLimitPushDownInsufficientBufferTest extends InitializedNullH
final GroupingEngine groupingEngine = new GroupingEngine(
druidProcessingConfig,
configSupplier,
bufferPool,
groupByResourcesReservationPool,
TestHelper.makeJsonMapper(),
new ObjectMapper(new SmileFactory()),
@ -346,7 +345,6 @@ public class GroupByLimitPushDownInsufficientBufferTest extends InitializedNullH
final GroupingEngine tooSmallEngine = new GroupingEngine(
tooSmallDruidProcessingConfig,
configSupplier,
bufferPool2,
tooSmallGroupByResourcesReservationPool,
TestHelper.makeJsonMapper(),
new ObjectMapper(new SmileFactory()),
@ -355,12 +353,14 @@ public class GroupByLimitPushDownInsufficientBufferTest extends InitializedNullH
groupByFactory = new GroupByQueryRunnerFactory(
groupingEngine,
new GroupByQueryQueryToolChest(groupingEngine, groupByResourcesReservationPool)
new GroupByQueryQueryToolChest(groupingEngine, groupByResourcesReservationPool),
bufferPool
);
tooSmallGroupByFactory = new GroupByQueryRunnerFactory(
tooSmallEngine,
new GroupByQueryQueryToolChest(tooSmallEngine, tooSmallGroupByResourcesReservationPool)
new GroupByQueryQueryToolChest(tooSmallEngine, tooSmallGroupByResourcesReservationPool),
bufferPool2
);
}

View File

@ -588,7 +588,6 @@ public class GroupByLimitPushDownMultiNodeMergeTest extends InitializedNullHandl
final GroupingEngine groupingEngineBroker = new GroupingEngine(
druidProcessingConfig,
configSupplier,
bufferPool,
groupByResourcesReservationPoolBroker,
TestHelper.makeJsonMapper(),
new ObjectMapper(new SmileFactory()),
@ -597,7 +596,6 @@ public class GroupByLimitPushDownMultiNodeMergeTest extends InitializedNullHandl
final GroupingEngine groupingEngineHistorical = new GroupingEngine(
druidProcessingConfig,
configSupplier,
bufferPool,
groupByResourcesReservationPoolHistorical,
TestHelper.makeJsonMapper(),
new ObjectMapper(new SmileFactory()),
@ -606,7 +604,6 @@ public class GroupByLimitPushDownMultiNodeMergeTest extends InitializedNullHandl
final GroupingEngine groupingEngineHistorical2 = new GroupingEngine(
druidProcessingConfig,
configSupplier,
bufferPool,
groupByResourcesReservationPoolHistorical2,
TestHelper.makeJsonMapper(),
new ObjectMapper(new SmileFactory()),
@ -615,17 +612,20 @@ public class GroupByLimitPushDownMultiNodeMergeTest extends InitializedNullHandl
groupByFactoryBroker = new GroupByQueryRunnerFactory(
groupingEngineBroker,
new GroupByQueryQueryToolChest(groupingEngineBroker, groupByResourcesReservationPoolBroker)
new GroupByQueryQueryToolChest(groupingEngineBroker, groupByResourcesReservationPoolBroker),
bufferPool
);
groupByFactoryHistorical = new GroupByQueryRunnerFactory(
groupingEngineHistorical,
new GroupByQueryQueryToolChest(groupingEngineHistorical, groupByResourcesReservationPoolHistorical)
new GroupByQueryQueryToolChest(groupingEngineHistorical, groupByResourcesReservationPoolHistorical),
bufferPool
);
groupByFactoryHistorical2 = new GroupByQueryRunnerFactory(
groupingEngineHistorical2,
new GroupByQueryQueryToolChest(groupingEngineHistorical2, groupByResourcesReservationPoolHistorical2)
new GroupByQueryQueryToolChest(groupingEngineHistorical2, groupByResourcesReservationPoolHistorical2),
bufferPool
);
}

View File

@ -245,7 +245,6 @@ public class GroupByMultiSegmentTest extends InitializedNullHandlingTest
final GroupingEngine groupingEngine = new GroupingEngine(
druidProcessingConfig,
configSupplier,
bufferPool,
groupByResourcesReservationPool,
TestHelper.makeJsonMapper(),
new ObjectMapper(new SmileFactory()),
@ -254,7 +253,8 @@ public class GroupByMultiSegmentTest extends InitializedNullHandlingTest
groupByFactory = new GroupByQueryRunnerFactory(
groupingEngine,
new GroupByQueryQueryToolChest(groupingEngine, groupByResourcesReservationPool)
new GroupByQueryQueryToolChest(groupingEngine, groupByResourcesReservationPool),
bufferPool
);
}

View File

@ -130,7 +130,6 @@ public class GroupByQueryMergeBufferTest extends InitializedNullHandlingTest
final GroupingEngine groupingEngine = new GroupingEngine(
PROCESSING_CONFIG,
configSupplier,
BUFFER_POOL,
groupByResourcesReservationPool,
TestHelper.makeJsonMapper(),
mapper,
@ -140,7 +139,7 @@ public class GroupByQueryMergeBufferTest extends InitializedNullHandlingTest
groupingEngine,
groupByResourcesReservationPool
);
return new GroupByQueryRunnerFactory(groupingEngine, toolChest);
return new GroupByQueryRunnerFactory(groupingEngine, toolChest, BUFFER_POOL);
}
private static final CloseableStupidPool<ByteBuffer> BUFFER_POOL = new CloseableStupidPool<>(

View File

@ -28,9 +28,7 @@ import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Lists;
import org.apache.druid.collections.BlockingPool;
import org.apache.druid.collections.DefaultBlockingPool;
import org.apache.druid.collections.NonBlockingPool;
import org.apache.druid.collections.SerializablePair;
import org.apache.druid.collections.StupidPool;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.data.input.Row;
import org.apache.druid.jackson.AggregatorsModule;
@ -1293,10 +1291,6 @@ public class GroupByQueryQueryToolChestTest extends InitializedNullHandlingTest
final Supplier<ByteBuffer> bufferSupplier =
() -> ByteBuffer.allocateDirect(processingConfig.intermediateComputeSizeBytes());
final NonBlockingPool<ByteBuffer> bufferPool = new StupidPool<>(
"GroupByQueryEngine-bufferPool",
bufferSupplier
);
final BlockingPool<ByteBuffer> mergeBufferPool = new DefaultBlockingPool<>(
bufferSupplier,
processingConfig.getNumMergeBuffers()
@ -1305,7 +1299,6 @@ public class GroupByQueryQueryToolChestTest extends InitializedNullHandlingTest
final GroupingEngine groupingEngine = new GroupingEngine(
processingConfig,
queryConfigSupplier,
bufferPool,
groupByResourcesReservationPool,
TestHelper.makeJsonMapper(),
new ObjectMapper(new SmileFactory()),

View File

@ -97,18 +97,19 @@ public class GroupByQueryRunnerFailureTest
)
{
final Supplier<GroupByQueryConfig> configSupplier = Suppliers.ofInstance(config);
GroupByResourcesReservationPool groupByResourcesReservationPool = new GroupByResourcesReservationPool(MERGE_BUFFER_POOL, config);
GroupByResourcesReservationPool groupByResourcesReservationPool =
new GroupByResourcesReservationPool(MERGE_BUFFER_POOL, config);
final GroupingEngine groupingEngine = new GroupingEngine(
DEFAULT_PROCESSING_CONFIG,
configSupplier,
BUFFER_POOL,
groupByResourcesReservationPool,
TestHelper.makeJsonMapper(),
mapper,
QueryRunnerTestHelper.NOOP_QUERYWATCHER
);
final GroupByQueryQueryToolChest toolChest = new GroupByQueryQueryToolChest(groupingEngine, groupByResourcesReservationPool);
return new GroupByQueryRunnerFactory(groupingEngine, toolChest);
final GroupByQueryQueryToolChest toolChest =
new GroupByQueryQueryToolChest(groupingEngine, groupByResourcesReservationPool);
return new GroupByQueryRunnerFactory(groupingEngine, toolChest, BUFFER_POOL);
}
private static final CloseableStupidPool<ByteBuffer> BUFFER_POOL = new CloseableStupidPool<>(

View File

@ -361,7 +361,6 @@ public class GroupByQueryRunnerTest extends InitializedNullHandlingTest
final GroupingEngine groupingEngine = new GroupingEngine(
processingConfig,
configSupplier,
bufferPools.getProcessingPool(),
groupByResourcesReservationPool,
mapper,
mapper,
@ -373,7 +372,7 @@ public class GroupByQueryRunnerTest extends InitializedNullHandlingTest
DefaultGroupByQueryMetricsFactory.instance(),
groupByResourcesReservationPool
);
return new GroupByQueryRunnerFactory(groupingEngine, toolChest);
return new GroupByQueryRunnerFactory(groupingEngine, toolChest, bufferPools.getProcessingPool());
}
@Parameterized.Parameters(name = "{0}")

View File

@ -292,7 +292,6 @@ public class NestedQueryPushDownTest extends InitializedNullHandlingTest
final GroupingEngine engine1 = new GroupingEngine(
druidProcessingConfig,
configSupplier,
bufferPool,
groupByResourcesReservationPool,
TestHelper.makeJsonMapper(),
new ObjectMapper(new SmileFactory()),
@ -301,7 +300,6 @@ public class NestedQueryPushDownTest extends InitializedNullHandlingTest
final GroupingEngine engine2 = new GroupingEngine(
druidProcessingConfig,
configSupplier,
bufferPool,
groupByResourcesReservationPool2,
TestHelper.makeJsonMapper(),
new ObjectMapper(new SmileFactory()),
@ -310,12 +308,14 @@ public class NestedQueryPushDownTest extends InitializedNullHandlingTest
groupByFactory = new GroupByQueryRunnerFactory(
engine1,
new GroupByQueryQueryToolChest(engine1, groupByResourcesReservationPool)
new GroupByQueryQueryToolChest(engine1, groupByResourcesReservationPool),
bufferPool
);
groupByFactory2 = new GroupByQueryRunnerFactory(
engine2,
new GroupByQueryQueryToolChest(engine2, groupByResourcesReservationPool2)
new GroupByQueryQueryToolChest(engine2, groupByResourcesReservationPool2),
bufferPool
);
}

View File

@ -178,7 +178,6 @@ public class UnnestGroupByQueryRunnerTest extends InitializedNullHandlingTest
final GroupingEngine groupingEngine = new GroupingEngine(
processingConfig,
configSupplier,
bufferPools.getProcessingPool(),
groupByResourcesReservationPool,
TestHelper.makeJsonMapper(),
mapper,
@ -186,7 +185,7 @@ public class UnnestGroupByQueryRunnerTest extends InitializedNullHandlingTest
);
final GroupByQueryQueryToolChest toolChest =
new GroupByQueryQueryToolChest(groupingEngine, groupByResourcesReservationPool);
return new GroupByQueryRunnerFactory(groupingEngine, toolChest);
return new GroupByQueryRunnerFactory(groupingEngine, toolChest, bufferPools.getProcessingPool());
}
@Parameterized.Parameters(name = "{0}")

View File

@ -62,6 +62,7 @@ import java.util.List;
public class CursorHolderPreaggTest extends InitializedNullHandlingTest
{
private CloseableStupidPool<ByteBuffer> bufferPool;
private GroupingEngine groupingEngine;
private TopNQueryEngine topNQueryEngine;
private TimeseriesQueryEngine timeseriesQueryEngine;
@ -75,18 +76,17 @@ public class CursorHolderPreaggTest extends InitializedNullHandlingTest
@Before
public void setup()
{
final CloseableStupidPool<ByteBuffer> pool = closer.closeLater(
bufferPool = closer.closeLater(
new CloseableStupidPool<>(
"CursorHolderPreaggTest-bufferPool",
() -> ByteBuffer.allocate(50000)
)
);
topNQueryEngine = new TopNQueryEngine(pool);
timeseriesQueryEngine = new TimeseriesQueryEngine(pool);
topNQueryEngine = new TopNQueryEngine(bufferPool);
timeseriesQueryEngine = new TimeseriesQueryEngine(bufferPool);
groupingEngine = new GroupingEngine(
new DruidProcessingConfig(),
GroupByQueryConfig::new,
pool,
new GroupByResourcesReservationPool(
closer.closeLater(
new CloseableDefaultBlockingPool<>(
@ -235,6 +235,7 @@ public class CursorHolderPreaggTest extends InitializedNullHandlingTest
query,
cursorFactory,
null,
bufferPool,
null
);
List<ResultRow> rows = results.toList();