Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Make partial aggregation adaptive #11011

Merged
merged 4 commits into from
Mar 11, 2022
Merged
Show file tree
Hide file tree
Changes from all 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 @@ -44,6 +44,7 @@
import static io.trino.spi.session.PropertyMetadata.doubleProperty;
import static io.trino.spi.session.PropertyMetadata.enumProperty;
import static io.trino.spi.session.PropertyMetadata.integerProperty;
import static io.trino.spi.session.PropertyMetadata.longProperty;
import static io.trino.spi.session.PropertyMetadata.stringProperty;
import static io.trino.spi.type.IntegerType.INTEGER;
import static io.trino.spi.type.TimeZoneKey.getTimeZoneKey;
Expand Down Expand Up @@ -156,6 +157,9 @@ public final class SystemSessionProperties
public static final String FAULT_TOLERANT_EXECUTION_TARGET_TASK_SPLIT_COUNT = "fault_tolerant_execution_target_task_split_count";
public static final String FAULT_TOLERANT_EXECUTION_MAX_TASK_SPLIT_COUNT = "fault_tolerant_execution_max_task_split_count";
public static final String FAULT_TOLERANT_EXECUTION_TASK_MEMORY = "fault_tolerant_execution_task_memory";
public static final String ADAPTIVE_PARTIAL_AGGREGATION_ENABLED = "adaptive_partial_aggregation_enabled";
public static final String ADAPTIVE_PARTIAL_AGGREGATION_MIN_ROWS = "adaptive_partial_aggregation_min_rows";
public static final String ADAPTIVE_PARTIAL_AGGREGATION_UNIQUE_ROWS_RATIO_THRESHOLD = "adaptive_partial_aggregation_unique_rows_ratio_threshold";

private final List<PropertyMetadata<?>> sessionProperties;

Expand Down Expand Up @@ -739,6 +743,21 @@ public SystemSessionProperties(
FAULT_TOLERANT_EXECUTION_TASK_MEMORY,
"Estimated amount of memory a single task will use when task level retries are used; value is used allocating nodes for tasks execution",
memoryManagerConfig.getFaultTolerantTaskMemory(),
false),
booleanProperty(
ADAPTIVE_PARTIAL_AGGREGATION_ENABLED,
"When enabled, partial aggregation might be adaptively turned off when it does not provide any performance gain",
optimizerConfig.isAdaptivePartialAggregationEnabled(),
false),
longProperty(
ADAPTIVE_PARTIAL_AGGREGATION_MIN_ROWS,
sopel39 marked this conversation as resolved.
Show resolved Hide resolved
"Minimum number of processed rows before partial aggregation might be adaptively turned off",
optimizerConfig.getAdaptivePartialAggregationMinRows(),
false),
doubleProperty(
ADAPTIVE_PARTIAL_AGGREGATION_UNIQUE_ROWS_RATIO_THRESHOLD,
"Ratio between aggregation output and input rows above which partial aggregation might be adaptively turned off",
optimizerConfig.getAdaptivePartialAggregationUniqueRowsRatioThreshold(),
false));
}

Expand Down Expand Up @@ -1330,4 +1349,19 @@ public static DataSize getFaultTolerantExecutionDefaultTaskMemory(Session sessio
{
return session.getSystemProperty(FAULT_TOLERANT_EXECUTION_TASK_MEMORY, DataSize.class);
}

public static boolean isAdaptivePartialAggregationEnabled(Session session)
{
return session.getSystemProperty(ADAPTIVE_PARTIAL_AGGREGATION_ENABLED, Boolean.class);
}

public static long getAdaptivePartialAggregationMinRows(Session session)
{
return session.getSystemProperty(ADAPTIVE_PARTIAL_AGGREGATION_MIN_ROWS, Long.class);
}

public static double getAdaptivePartialAggregationUniqueRowsRatioThreshold(Session session)
{
return session.getSystemProperty(ADAPTIVE_PARTIAL_AGGREGATION_UNIQUE_ROWS_RATIO_THRESHOLD, Double.class);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -13,13 +13,21 @@
*/
package io.trino.operator;

import javax.annotation.Nullable;

import static java.util.Objects.requireNonNull;

public final class CompletedWork<T>
implements Work<T>
{
@Nullable
private final T result;
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

mark result as @Nullable


public CompletedWork()
{
this.result = null;
}

public CompletedWork(T value)
{
this.result = requireNonNull(value);
Expand All @@ -31,6 +39,7 @@ public boolean process()
return true;
}

@Nullable
@Override
public T getResult()
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,8 @@
import io.trino.operator.aggregation.builder.HashAggregationBuilder;
import io.trino.operator.aggregation.builder.InMemoryHashAggregationBuilder;
import io.trino.operator.aggregation.builder.SpillableHashAggregationBuilder;
import io.trino.operator.aggregation.partial.PartialAggregationController;
import io.trino.operator.aggregation.partial.SkipAggregationBuilder;
import io.trino.operator.scalar.CombineHashFunction;
import io.trino.spi.Page;
import io.trino.spi.PageBuilder;
Expand All @@ -36,6 +38,7 @@
import java.util.List;
import java.util.Optional;

import static com.google.common.base.Preconditions.checkArgument;
import static com.google.common.base.Preconditions.checkState;
import static io.airlift.units.DataSize.Unit.MEGABYTE;
import static io.trino.operator.aggregation.builder.InMemoryHashAggregationBuilder.toTypes;
Expand Down Expand Up @@ -70,6 +73,7 @@ public static class HashAggregationOperatorFactory
private final SpillerFactory spillerFactory;
private final JoinCompiler joinCompiler;
private final BlockTypeOperators blockTypeOperators;
private final Optional<PartialAggregationController> partialAggregationController;

private boolean closed;

Expand All @@ -87,7 +91,8 @@ public HashAggregationOperatorFactory(
int expectedGroups,
Optional<DataSize> maxPartialMemory,
JoinCompiler joinCompiler,
BlockTypeOperators blockTypeOperators)
BlockTypeOperators blockTypeOperators,
Optional<PartialAggregationController> partialAggregationController)
{
this(operatorId,
planNodeId,
Expand All @@ -108,7 +113,8 @@ public HashAggregationOperatorFactory(
throw new UnsupportedOperationException();
},
joinCompiler,
blockTypeOperators);
blockTypeOperators,
partialAggregationController);
}

public HashAggregationOperatorFactory(
Expand All @@ -128,7 +134,8 @@ public HashAggregationOperatorFactory(
DataSize unspillMemoryLimit,
SpillerFactory spillerFactory,
JoinCompiler joinCompiler,
BlockTypeOperators blockTypeOperators)
BlockTypeOperators blockTypeOperators,
Optional<PartialAggregationController> partialAggregationController)
{
this(operatorId,
planNodeId,
Expand All @@ -147,7 +154,8 @@ public HashAggregationOperatorFactory(
DataSize.succinctBytes((long) (unspillMemoryLimit.toBytes() * MERGE_WITH_MEMORY_RATIO)),
spillerFactory,
joinCompiler,
blockTypeOperators);
blockTypeOperators,
partialAggregationController);
}

@VisibleForTesting
Expand All @@ -169,7 +177,8 @@ public HashAggregationOperatorFactory(
DataSize memoryLimitForMergeWithMemory,
SpillerFactory spillerFactory,
JoinCompiler joinCompiler,
BlockTypeOperators blockTypeOperators)
BlockTypeOperators blockTypeOperators,
Optional<PartialAggregationController> partialAggregationController)
{
this.operatorId = operatorId;
this.planNodeId = requireNonNull(planNodeId, "planNodeId is null");
Expand All @@ -189,6 +198,7 @@ public HashAggregationOperatorFactory(
this.spillerFactory = requireNonNull(spillerFactory, "spillerFactory is null");
this.joinCompiler = requireNonNull(joinCompiler, "joinCompiler is null");
this.blockTypeOperators = requireNonNull(blockTypeOperators, "blockTypeOperators is null");
this.partialAggregationController = requireNonNull(partialAggregationController, "partialAggregationController is null");
}

@Override
Expand All @@ -214,7 +224,8 @@ public Operator createOperator(DriverContext driverContext)
memoryLimitForMergeWithMemory,
spillerFactory,
joinCompiler,
blockTypeOperators);
blockTypeOperators,
partialAggregationController);
return hashAggregationOperator;
}

Expand Down Expand Up @@ -245,11 +256,13 @@ public OperatorFactory duplicate()
memoryLimitForMergeWithMemory,
spillerFactory,
joinCompiler,
blockTypeOperators);
blockTypeOperators,
partialAggregationController.map(PartialAggregationController::duplicate));
}
}

private final OperatorContext operatorContext;
private final Optional<PartialAggregationController> partialAggregationController;
private final List<Type> groupByTypes;
private final List<Integer> groupByChannels;
private final List<Integer> globalAggregationGroupIds;
Expand Down Expand Up @@ -279,6 +292,8 @@ public OperatorFactory duplicate()

// for yield when memory is not available
private Work<?> unfinishedWork;
private long numberOfInputRowsProcessed;
private long numberOfUniqueRowsProduced;

private HashAggregationOperator(
OperatorContext operatorContext,
Expand All @@ -297,12 +312,15 @@ private HashAggregationOperator(
DataSize memoryLimitForMergeWithMemory,
SpillerFactory spillerFactory,
JoinCompiler joinCompiler,
BlockTypeOperators blockTypeOperators)
BlockTypeOperators blockTypeOperators,
Optional<PartialAggregationController> partialAggregationController)
{
this.operatorContext = requireNonNull(operatorContext, "operatorContext is null");
this.partialAggregationController = requireNonNull(partialAggregationController, "partialAggregationControl is null");
requireNonNull(step, "step is null");
requireNonNull(aggregatorFactories, "aggregatorFactories is null");
requireNonNull(operatorContext, "operatorContext is null");
checkArgument(partialAggregationController.isEmpty() || step.isOutputPartial(), "partialAggregationController should be present only for partial aggregation");

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: restore newline

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

restored after the checkArgument

this.groupByTypes = ImmutableList.copyOf(groupByTypes);
this.groupByChannels = ImmutableList.copyOf(groupByChannels);
Expand Down Expand Up @@ -368,8 +386,14 @@ public void addInput(Page page)
inputProcessed = true;

if (aggregationBuilder == null) {
// TODO: We ignore spillEnabled here if any aggregate has ORDER BY clause or DISTINCT because they are not yet implemented for spilling.
if (step.isOutputPartial() || !spillEnabled || !isSpillable()) {
boolean partialAggregationDisabled = partialAggregationController
.map(PartialAggregationController::isPartialAggregationDisabled)
.orElse(false);
if (step.isOutputPartial() && partialAggregationDisabled) {
aggregationBuilder = new SkipAggregationBuilder(groupByChannels, hashChannel, aggregatorFactories, memoryContext);
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

future improvement. It would be great to actually collect metrics

  • how many pages were processed via skip aggregation builder
  • how many flushes there were for PA.
  • what average row count per flush
  • etc..

This can be returned via io.trino.operator.OperatorContext#setLatestMetrics

@lukasz-stec Maybe create an issue for that?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Good idea. This would allow easier monitoring of the adaptation.
#11376 created.

}
else if (step.isOutputPartial() || !spillEnabled || !isSpillable()) {
sopel39 marked this conversation as resolved.
Show resolved Hide resolved
// TODO: We ignore spillEnabled here if any aggregate has ORDER BY clause or DISTINCT because they are not yet implemented for spilling.
aggregationBuilder = new InMemoryHashAggregationBuilder(
aggregatorFactories,
step,
Expand Down Expand Up @@ -418,6 +442,7 @@ public void addInput(Page page)
unfinishedWork = null;
}
aggregationBuilder.updateMemory();
numberOfInputRowsProcessed += page.getPositionCount();
}

private boolean isSpillable()
Expand Down Expand Up @@ -490,7 +515,9 @@ public Page getOutput()
return null;
}

return outputPages.getResult();
Page result = outputPages.getResult();
numberOfUniqueRowsProduced += result.getPositionCount();
return result;
}

@Override
Expand All @@ -516,6 +543,10 @@ private void closeAggregationBuilder()
aggregationBuilder = null;
}
memoryContext.setBytes(0);
partialAggregationController.ifPresent(
controller -> controller.onFlush(numberOfInputRowsProcessed, numberOfUniqueRowsProduced));
numberOfInputRowsProcessed = 0;
numberOfUniqueRowsProduced = 0;
}

private Page getGlobalAggregationOutput()
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,73 @@
/*
* Licensed 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 io.trino.operator.aggregation.partial;

import io.trino.operator.HashAggregationOperator;

/**
* Controls whenever partial aggregation is enabled across all {@link HashAggregationOperator}s
* for a particular plan node on a single node.
* Partial aggregation is disabled once enough rows has been processed ({@link #minNumberOfRowsProcessed})
* and the ratio between output(unique) and input rows is too high (> {@link #uniqueRowsRatioThreshold}).
* TODO https://github.com/trinodb/trino/issues/11361 add support to adaptively re-enable partial aggregation.
* <p>
* The class is thread safe and objects of this class are used potentially by multiple threads/drivers simultaneously.
* Different threads either:
* - modify fields via synchronized {@link #onFlush}.
* - read volatile {@link #partialAggregationDisabled} (volatile here gives visibility).
*/
public class PartialAggregationController
{
private final long minNumberOfRowsProcessed;
private final double uniqueRowsRatioThreshold;

private volatile boolean partialAggregationDisabled;
private long totalRowProcessed;
private long totalUniqueRowsProduced;

public PartialAggregationController(long minNumberOfRowsProcessedToDisable, double uniqueRowsRatioThreshold)
{
this.minNumberOfRowsProcessed = minNumberOfRowsProcessedToDisable;
this.uniqueRowsRatioThreshold = uniqueRowsRatioThreshold;
}

public boolean isPartialAggregationDisabled()
{
return partialAggregationDisabled;
}

public synchronized void onFlush(long rowsProcessed, long uniqueRowsProduced)
{
if (partialAggregationDisabled) {
return;
}

totalRowProcessed += rowsProcessed;
totalUniqueRowsProduced += uniqueRowsProduced;
if (shouldDisablePartialAggregation()) {
partialAggregationDisabled = true;
}
}

private boolean shouldDisablePartialAggregation()
{
return totalRowProcessed >= minNumberOfRowsProcessed
&& ((double) totalUniqueRowsProduced / totalRowProcessed) > uniqueRowsRatioThreshold;
}

public PartialAggregationController duplicate()
{
return new PartialAggregationController(minNumberOfRowsProcessed, uniqueRowsRatioThreshold);
}
}
Loading