Skip to content
Merged
Show file tree
Hide file tree
Changes from 5 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
import io.airlift.concurrent.MoreFutures;
import io.airlift.log.Logger;
import io.airlift.units.DataSize;
import io.trino.memory.context.CoarseGrainLocalMemoryContext;
import io.trino.memory.context.LocalMemoryContext;
import io.trino.operator.DriverContext;
import io.trino.operator.HashArraySizeSupplier;
Expand Down Expand Up @@ -57,6 +58,7 @@
import static io.airlift.concurrent.MoreFutures.checkSuccess;
import static io.airlift.concurrent.MoreFutures.getDone;
import static io.airlift.units.DataSize.succinctBytes;
import static io.trino.memory.context.CoarseGrainLocalMemoryContext.DEFAULT_GRANULARITY;
import static java.lang.String.format;
import static java.util.Objects.requireNonNull;

Expand Down Expand Up @@ -149,7 +151,8 @@ public HashBuilderOperator createOperator(DriverContext driverContext)
pagesIndexFactory,
spillEnabled,
singleStreamSpillerFactory,
hashArraySizeSupplier);
hashArraySizeSupplier,
DEFAULT_GRANULARITY);
}

@Override
Expand Down Expand Up @@ -235,6 +238,7 @@ public enum State
private Optional<SingleStreamSpiller> spiller = Optional.empty();
private ListenableFuture<DataSize> spillInProgress = immediateFuture(DataSize.ofBytes(0));
private Optional<ListenableFuture<List<Page>>> unspillInProgress = Optional.empty();
private boolean unspilledPagesAdded;
@Nullable
private LookupSourceSupplier lookupSourceSupplier;
private OptionalLong lookupSourceChecksum = OptionalLong.empty();
Expand All @@ -255,7 +259,8 @@ public HashBuilderOperator(
PagesIndex.Factory pagesIndexFactory,
boolean spillEnabled,
SingleStreamSpillerFactory singleStreamSpillerFactory,
HashArraySizeSupplier hashArraySizeSupplier)
HashArraySizeSupplier hashArraySizeSupplier,
long memorySyncGranularity)
{
requireNonNull(pagesIndexFactory, "pagesIndexFactory is null");

Expand All @@ -264,8 +269,8 @@ public HashBuilderOperator(
this.filterFunctionFactory = filterFunctionFactory;
this.sortChannel = sortChannel;
this.searchFunctionFactories = searchFunctionFactories;
this.localUserMemoryContext = operatorContext.localUserMemoryContext();
this.localRevocableMemoryContext = operatorContext.localRevocableMemoryContext();
this.localUserMemoryContext = new CoarseGrainLocalMemoryContext(operatorContext.localUserMemoryContext(), memorySyncGranularity);
this.localRevocableMemoryContext = new CoarseGrainLocalMemoryContext(operatorContext.localRevocableMemoryContext(), memorySyncGranularity);

this.index = pagesIndexFactory.newPagesIndex(lookupSourceFactory.getTypes(), expectedPositions);
this.lookupSourceFactory = lookupSourceFactory;
Expand Down Expand Up @@ -373,6 +378,7 @@ public ListenableFuture<Void> startMemoryRevoke()
long indexSizeAfterCompaction = index.getEstimatedSize().toBytes();
if (indexSizeAfterCompaction < indexSizeBeforeCompaction * INDEX_COMPACTION_ON_REVOCATION_TARGET) {
finishMemoryRevoke = Optional.of(() -> {});
localRevocableMemoryContext.setBytes(indexSizeAfterCompaction);
return immediateVoidFuture();
}

Expand Down Expand Up @@ -495,6 +501,24 @@ private void finishInput()
return;
}

long memoryRequired = index.getEstimatedMemoryRequiredToCreateLookupSource(
hashArraySizeSupplier,
sortChannel,
hashChannels);

ListenableFuture<Void> reserved;
if (spillEnabled) {
reserved = localRevocableMemoryContext.setBytes(memoryRequired);
}
else {
reserved = localUserMemoryContext.setBytes(memoryRequired);
}

if (!reserved.isDone()) {
// wait for memory
return;
}

LookupSourceSupplier partition = buildLookupSource();
if (spillEnabled) {
localRevocableMemoryContext.setBytes(partition.get().getInMemorySizeInBytes());
Expand Down Expand Up @@ -545,7 +569,10 @@ private void unspillLookupSourceIfRequested()
verify(unspillInProgress.isEmpty());

long spilledPagesInMemorySize = getSpiller().getSpilledPagesInMemorySize();
localUserMemoryContext.setBytes(spilledPagesInMemorySize + index.getEstimatedSize().toBytes());
if (!localUserMemoryContext.setBytes(spilledPagesInMemorySize + index.getEstimatedSize().toBytes()).isDone()) {
// wait for memory
return;
}
long unspillStartNanos = System.nanoTime();
unspillInProgress = Optional.of(getSpiller().getAllSpilledPages());
addSuccessCallback(unspillInProgress.get(), ignored -> {
Expand All @@ -554,39 +581,53 @@ private void unspillLookupSourceIfRequested()
});

state = State.INPUT_UNSPILLING;
unspilledPagesAdded = false;
}

private void finishLookupSourceUnspilling()
{
checkState(state == State.INPUT_UNSPILLING);
if (!unspillInProgress.get().isDone()) {
// Pages have not been unspilled yet.
return;
}

// Use Queue so that Pages already consumed by Index are not retained by us.
Queue<Page> pages = new ArrayDeque<>(getDone(unspillInProgress.get()));
unspillInProgress = Optional.empty();
long sizeOfUnspilledPages = pages.stream()
.mapToLong(Page::getSizeInBytes)
.sum();
long retainedSizeOfUnspilledPages = pages.stream()
.mapToLong(Page::getRetainedSizeInBytes)
.sum();
log.debug(
"Unspilling for operator %s, unspilled partition %d, sizeOfUnspilledPages %s, retainedSizeOfUnspilledPages %s",
operatorContext,
partitionIndex,
succinctBytes(sizeOfUnspilledPages),
succinctBytes(retainedSizeOfUnspilledPages));
localUserMemoryContext.setBytes(retainedSizeOfUnspilledPages + index.getEstimatedSize().toBytes());

while (!pages.isEmpty()) {
Page next = pages.remove();
index.addPage(next);
// There is no attempt to compact index, since unspilled pages are unlikely to have blocks with retained size > logical size.
retainedSizeOfUnspilledPages -= next.getRetainedSizeInBytes();
if (!unspilledPagesAdded) {
if (!unspillInProgress.get().isDone()) {
// Pages have not been unspilled yet.
return;
}

Queue<Page> pages = new ArrayDeque<>(getDone(unspillInProgress.get()));
unspillInProgress = Optional.empty();
long sizeOfUnspilledPages = pages.stream()
.mapToLong(Page::getSizeInBytes)
.sum();
long retainedSizeOfUnspilledPages = pages.stream()
.mapToLong(Page::getRetainedSizeInBytes)
.sum();
log.debug(
"Unspilling for operator %s, unspilled partition %d, sizeOfUnspilledPages %s, retainedSizeOfUnspilledPages %s",
operatorContext,
partitionIndex,
succinctBytes(sizeOfUnspilledPages),
succinctBytes(retainedSizeOfUnspilledPages));
localUserMemoryContext.setBytes(retainedSizeOfUnspilledPages + index.getEstimatedSize().toBytes());

while (!pages.isEmpty()) {
Page next = pages.remove();
index.addPage(next);
// There is no attempt to compact index, since unspilled pages are unlikely to have blocks with retained size > logical size.
retainedSizeOfUnspilledPages -= next.getRetainedSizeInBytes();
localUserMemoryContext.setBytes(retainedSizeOfUnspilledPages + index.getEstimatedSize().toBytes());
}

unspilledPagesAdded = true;
}

ListenableFuture<Void> reserved = localUserMemoryContext.setBytes(index.getEstimatedMemoryRequiredToCreateLookupSource(
hashArraySizeSupplier,
sortChannel,
hashChannels));
if (!reserved.isDone()) {
// Wait for memory
return;
}

LookupSourceSupplier partition = buildLookupSource();
Expand Down
Loading