Skip to content

Commit 6765613

Browse files
committed
Limit total size of dynamic filter collected per operator
1 parent 0cc6959 commit 6765613

File tree

6 files changed

+323
-79
lines changed

6 files changed

+323
-79
lines changed

core/trino-main/src/main/java/io/trino/execution/DynamicFilterConfig.java

Lines changed: 57 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -23,6 +23,7 @@
2323
import javax.validation.constraints.Min;
2424

2525
import static io.airlift.units.DataSize.Unit.KILOBYTE;
26+
import static io.airlift.units.DataSize.Unit.MEGABYTE;
2627

2728
@DefunctConfig({
2829
"dynamic-filtering-max-per-driver-row-count",
@@ -42,16 +43,20 @@ public class DynamicFilterConfig
4243
private int smallBroadcastMaxDistinctValuesPerDriver = 200;
4344
private DataSize smallBroadcastMaxSizePerDriver = DataSize.of(20, KILOBYTE);
4445
private int smallBroadcastRangeRowLimitPerDriver = 400;
46+
private DataSize smallBroadcastMaxSizePerOperator = DataSize.of(200, KILOBYTE);
4547
private int smallPartitionedMaxDistinctValuesPerDriver = 20;
4648
private DataSize smallPartitionedMaxSizePerDriver = DataSize.of(10, KILOBYTE);
4749
private int smallPartitionedRangeRowLimitPerDriver = 100;
50+
private DataSize smallPartitionedMaxSizePerOperator = DataSize.of(100, KILOBYTE);
4851

4952
private int largeBroadcastMaxDistinctValuesPerDriver = 5_000;
5053
private DataSize largeBroadcastMaxSizePerDriver = DataSize.of(500, KILOBYTE);
5154
private int largeBroadcastRangeRowLimitPerDriver = 10_000;
55+
private DataSize largeBroadcastMaxSizePerOperator = DataSize.of(5, MEGABYTE);
5256
private int largePartitionedMaxDistinctValuesPerDriver = 500;
5357
private DataSize largePartitionedMaxSizePerDriver = DataSize.of(50, KILOBYTE);
5458
private int largePartitionedRangeRowLimitPerDriver = 1_000;
59+
private DataSize largePartitionedMaxSizePerOperator = DataSize.of(500, KILOBYTE);
5560

5661
public boolean isEnableDynamicFiltering()
5762
{
@@ -143,6 +148,19 @@ public DynamicFilterConfig setSmallBroadcastRangeRowLimitPerDriver(int smallBroa
143148
return this;
144149
}
145150

151+
@MaxDataSize("10MB")
152+
public DataSize getSmallBroadcastMaxSizePerOperator()
153+
{
154+
return smallBroadcastMaxSizePerOperator;
155+
}
156+
157+
@Config("dynamic-filtering.small-broadcast.max-size-per-operator")
158+
public DynamicFilterConfig setSmallBroadcastMaxSizePerOperator(DataSize smallBroadcastMaxSizePerOperator)
159+
{
160+
this.smallBroadcastMaxSizePerOperator = smallBroadcastMaxSizePerOperator;
161+
return this;
162+
}
163+
146164
@Min(0)
147165
public int getSmallPartitionedMaxDistinctValuesPerDriver()
148166
{
@@ -182,6 +200,19 @@ public DynamicFilterConfig setSmallPartitionedRangeRowLimitPerDriver(int smallPa
182200
return this;
183201
}
184202

203+
@MaxDataSize("10MB")
204+
public DataSize getSmallPartitionedMaxSizePerOperator()
205+
{
206+
return smallPartitionedMaxSizePerOperator;
207+
}
208+
209+
@Config("dynamic-filtering.small-partitioned.max-size-per-operator")
210+
public DynamicFilterConfig setSmallPartitionedMaxSizePerOperator(DataSize smallPartitionedMaxSizePerOperator)
211+
{
212+
this.smallPartitionedMaxSizePerOperator = smallPartitionedMaxSizePerOperator;
213+
return this;
214+
}
215+
185216
@Min(0)
186217
public int getLargeBroadcastMaxDistinctValuesPerDriver()
187218
{
@@ -221,6 +252,19 @@ public DynamicFilterConfig setLargeBroadcastRangeRowLimitPerDriver(int largeBroa
221252
return this;
222253
}
223254

255+
@MaxDataSize("100MB")
256+
public DataSize getLargeBroadcastMaxSizePerOperator()
257+
{
258+
return largeBroadcastMaxSizePerOperator;
259+
}
260+
261+
@Config("dynamic-filtering.large-broadcast.max-size-per-operator")
262+
public DynamicFilterConfig setLargeBroadcastMaxSizePerOperator(DataSize largeBroadcastMaxSizePerOperator)
263+
{
264+
this.largeBroadcastMaxSizePerOperator = largeBroadcastMaxSizePerOperator;
265+
return this;
266+
}
267+
224268
@Min(0)
225269
public int getLargePartitionedMaxDistinctValuesPerDriver()
226270
{
@@ -259,4 +303,17 @@ public DynamicFilterConfig setLargePartitionedRangeRowLimitPerDriver(int largePa
259303
this.largePartitionedRangeRowLimitPerDriver = largePartitionedRangeRowLimitPerDriver;
260304
return this;
261305
}
306+
307+
@MaxDataSize("50MB")
308+
public DataSize getLargePartitionedMaxSizePerOperator()
309+
{
310+
return largePartitionedMaxSizePerOperator;
311+
}
312+
313+
@Config("dynamic-filtering.large-partitioned.max-size-per-operator")
314+
public DynamicFilterConfig setLargePartitionedMaxSizePerOperator(DataSize largePartitionedMaxSizePerOperator)
315+
{
316+
this.largePartitionedMaxSizePerOperator = largePartitionedMaxSizePerOperator;
317+
return this;
318+
}
262319
}

core/trino-main/src/main/java/io/trino/sql/planner/LocalDynamicFilterConsumer.java

Lines changed: 95 additions & 58 deletions
Original file line numberDiff line numberDiff line change
@@ -13,105 +13,128 @@
1313
*/
1414
package io.trino.sql.planner;
1515

16+
import com.google.common.collect.ImmutableList;
1617
import com.google.common.collect.ImmutableMap;
18+
import io.airlift.units.DataSize;
1719
import io.trino.spi.predicate.Domain;
1820
import io.trino.spi.predicate.TupleDomain;
1921
import io.trino.spi.type.Type;
2022
import io.trino.sql.planner.plan.DynamicFilterId;
2123
import io.trino.sql.planner.plan.JoinNode;
2224
import io.trino.sql.planner.plan.PlanNode;
2325

24-
import javax.annotation.Nullable;
2526
import javax.annotation.concurrent.GuardedBy;
2627

27-
import java.util.ArrayList;
2828
import java.util.HashMap;
2929
import java.util.List;
3030
import java.util.Map;
31+
import java.util.Queue;
3132
import java.util.Set;
33+
import java.util.concurrent.ConcurrentLinkedQueue;
3234
import java.util.function.Consumer;
3335

3436
import static com.google.common.base.MoreObjects.toStringHelper;
3537
import static com.google.common.base.Preconditions.checkArgument;
3638
import static com.google.common.base.Preconditions.checkState;
3739
import static com.google.common.base.Verify.verify;
3840
import static com.google.common.collect.ImmutableMap.toImmutableMap;
41+
import static io.trino.spi.predicate.TupleDomain.columnWiseUnion;
3942
import static java.util.Objects.requireNonNull;
4043
import static java.util.function.Function.identity;
4144

4245
public class LocalDynamicFilterConsumer
4346
implements DynamicFilterSourceConsumer
4447
{
45-
private static final int PARTITION_COUNT_INITIAL_VALUE = -1;
4648
// Mapping from dynamic filter ID to its build channel indices.
4749
private final Map<DynamicFilterId, Integer> buildChannels;
48-
4950
// Mapping from dynamic filter ID to its build channel type.
5051
private final Map<DynamicFilterId, Type> filterBuildTypes;
51-
5252
private final List<Consumer<Map<DynamicFilterId, Domain>>> collectors;
53+
private final long domainSizeLimitInBytes;
5354

5455
// Number of build-side partitions to be collected, must be provided by setPartitionCount
5556
@GuardedBy("this")
56-
private int expectedPartitionCount = PARTITION_COUNT_INITIAL_VALUE;
57-
57+
private Integer expectedPartitionCount;
5858
@GuardedBy("this")
59-
private boolean collected;
60-
61-
// The resulting predicates from each build-side partition.
62-
@Nullable
59+
private int collectedPartitionCount;
6360
@GuardedBy("this")
64-
private List<TupleDomain<DynamicFilterId>> partitions;
61+
private volatile boolean collected;
6562

66-
public LocalDynamicFilterConsumer(
67-
Map<DynamicFilterId, Integer> buildChannels,
68-
Map<DynamicFilterId, Type> filterBuildTypes,
69-
List<Consumer<Map<DynamicFilterId, Domain>>> collectors)
63+
private final Queue<TupleDomain<DynamicFilterId>> summaryDomains = new ConcurrentLinkedQueue<>();
64+
65+
public LocalDynamicFilterConsumer(Map<DynamicFilterId, Integer> buildChannels, Map<DynamicFilterId, Type> filterBuildTypes, List<Consumer<Map<DynamicFilterId, Domain>>> collectors, DataSize domainSizeLimit)
7066
{
7167
this.buildChannels = requireNonNull(buildChannels, "buildChannels is null");
7268
this.filterBuildTypes = requireNonNull(filterBuildTypes, "filterBuildTypes is null");
7369
verify(buildChannels.keySet().equals(filterBuildTypes.keySet()), "filterBuildTypes and buildChannels must have same keys");
74-
7570
requireNonNull(collectors, "collectors is null");
7671
checkArgument(!collectors.isEmpty(), "collectors is empty");
77-
this.collectors = collectors;
78-
this.partitions = new ArrayList<>();
72+
this.collectors = ImmutableList.copyOf(collectors);
73+
this.domainSizeLimitInBytes = requireNonNull(domainSizeLimit, "domainSizeLimit is null").toBytes();
7974
}
8075

8176
@Override
82-
public void addPartition(TupleDomain<DynamicFilterId> tupleDomain)
77+
public void addPartition(TupleDomain<DynamicFilterId> domain)
8378
{
79+
if (collected) {
80+
return;
81+
}
82+
83+
summaryDomains.add(domain);
84+
// Operators collecting dynamic filters tend to finish all at the same time
85+
// when filters are collected right before the HashBuilderOperator.
86+
// To avoid multiple task executor threads being blocked on waiting
87+
// for each other when collecting the filters run the heavy union operation
88+
// outside the lock.
89+
unionSummaryDomains();
90+
8491
TupleDomain<DynamicFilterId> result;
8592
synchronized (this) {
93+
verify(expectedPartitionCount == null || collectedPartitionCount < expectedPartitionCount);
94+
8695
if (collected) {
96+
summaryDomains.clear();
8797
return;
8898
}
89-
requireNonNull(partitions, "partitions is null");
90-
// Called concurrently by each DynamicFilterSourceOperator instance (when collection is over).
91-
verify(expectedPartitionCount == PARTITION_COUNT_INITIAL_VALUE || partitions.size() < expectedPartitionCount);
92-
// NOTE: may result in a bit more relaxed constraint if there are multiple columns and multiple rows.
93-
// See the comment at TupleDomain::columnWiseUnion() for more details.
94-
partitions.add(tupleDomain);
95-
if (tupleDomain.isAll()) {
96-
result = tupleDomain;
99+
collectedPartitionCount++;
100+
101+
boolean allPartitionsCollected = expectedPartitionCount != null && collectedPartitionCount == expectedPartitionCount;
102+
if (allPartitionsCollected) {
103+
// run final compaction as previous concurrent compactions may have left more than a single domain
104+
unionSummaryDomains();
97105
}
98-
else if (partitions.size() == expectedPartitionCount) {
99-
// No more partitions are left to be processed.
100-
if (partitions.isEmpty()) {
101-
result = TupleDomain.none();
106+
107+
boolean sizeLimitExceeded = false;
108+
TupleDomain<DynamicFilterId> summary = summaryDomains.poll();
109+
// summary can be null as another concurrent summary compaction may be running
110+
if (summary != null) {
111+
if (summary.getRetainedSizeInBytes(DynamicFilterId::getRetainedSizeInBytes) > domainSizeLimitInBytes) {
112+
summary = summary.simplify(1);
102113
}
103-
else {
104-
result = TupleDomain.columnWiseUnion(partitions);
114+
if (summary.getRetainedSizeInBytes(DynamicFilterId::getRetainedSizeInBytes) > domainSizeLimitInBytes) {
115+
sizeLimitExceeded = true;
105116
}
117+
summaryDomains.add(summary);
106118
}
107-
else {
119+
120+
if (!allPartitionsCollected && !sizeLimitExceeded && !domain.isAll()) {
108121
return;
109122
}
123+
124+
if (sizeLimitExceeded || domain.isAll()) {
125+
summaryDomains.clear();
126+
result = TupleDomain.all();
127+
}
128+
else {
129+
verify(expectedPartitionCount != null && collectedPartitionCount == expectedPartitionCount);
130+
verify(summaryDomains.size() == 1);
131+
result = summaryDomains.poll();
132+
verify(result != null);
133+
}
110134
collected = true;
111-
partitions = null;
112135
}
113136

114-
notifyConsumers(result);
137+
collectors.forEach(collector -> collector.accept(convertTupleDomain(result)));
115138
}
116139

117140
@Override
@@ -122,33 +145,43 @@ public void setPartitionCount(int partitionCount)
122145
if (collected) {
123146
return;
124147
}
125-
checkState(expectedPartitionCount == PARTITION_COUNT_INITIAL_VALUE, "setPartitionCount should be called only once");
126-
requireNonNull(partitions, "partitions is null");
148+
checkState(expectedPartitionCount == null, "setPartitionCount should be called only once");
127149
expectedPartitionCount = partitionCount;
128-
if (partitions.size() == expectedPartitionCount) {
129-
// No more partitions are left to be processed.
130-
if (partitions.isEmpty()) {
131-
result = TupleDomain.none();
132-
}
133-
else {
134-
result = TupleDomain.columnWiseUnion(partitions);
135-
}
136-
collected = true;
137-
partitions = null;
150+
if (collectedPartitionCount < expectedPartitionCount) {
151+
return;
152+
}
153+
if (partitionCount == 0) {
154+
result = TupleDomain.all();
138155
}
139156
else {
140-
return;
157+
// run final compaction as previous concurrent compactions may have left more than a single domain
158+
unionSummaryDomains();
159+
verify(summaryDomains.size() == 1);
160+
result = summaryDomains.poll();
161+
verify(result != null);
141162
}
163+
collected = true;
142164
}
143165

144-
notifyConsumers(result);
166+
collectors.forEach(collector -> collector.accept(convertTupleDomain(result)));
145167
}
146168

147-
private void notifyConsumers(TupleDomain<DynamicFilterId> result)
169+
private void unionSummaryDomains()
148170
{
149-
requireNonNull(result, "result is null");
150-
Map<DynamicFilterId, Domain> dynamicFilterDomains = convertTupleDomain(result);
151-
collectors.forEach(consumer -> consumer.accept(dynamicFilterDomains));
171+
while (true) {
172+
// This method is called every time a new domain is added to the summaryDomains queue.
173+
// In a normal situation (when there's no race) there should be no more than 2 domains in the queue.
174+
TupleDomain<DynamicFilterId> first = summaryDomains.poll();
175+
if (first == null) {
176+
return;
177+
}
178+
TupleDomain<DynamicFilterId> second = summaryDomains.poll();
179+
if (second == null) {
180+
summaryDomains.add(first);
181+
return;
182+
}
183+
summaryDomains.add(columnWiseUnion(first, second));
184+
}
152185
}
153186

154187
private Map<DynamicFilterId, Domain> convertTupleDomain(TupleDomain<DynamicFilterId> result)
@@ -169,7 +202,8 @@ public static LocalDynamicFilterConsumer create(
169202
JoinNode planNode,
170203
List<Type> buildSourceTypes,
171204
Set<DynamicFilterId> collectedFilters,
172-
List<Consumer<Map<DynamicFilterId, Domain>>> collectors)
205+
List<Consumer<Map<DynamicFilterId, Domain>>> collectors,
206+
DataSize domainSizeLimit)
173207
{
174208
checkArgument(!planNode.getDynamicFilters().isEmpty(), "Join node dynamicFilters is empty.");
175209
checkArgument(!collectedFilters.isEmpty(), "Collected dynamic filters set is empty");
@@ -193,7 +227,7 @@ public static LocalDynamicFilterConsumer create(
193227
.collect(toImmutableMap(
194228
Map.Entry::getKey,
195229
entry -> buildSourceTypes.get(entry.getValue())));
196-
return new LocalDynamicFilterConsumer(buildChannels, filterBuildTypes, collectors);
230+
return new LocalDynamicFilterConsumer(buildChannels, filterBuildTypes, collectors, domainSizeLimit);
197231
}
198232

199233
public Map<DynamicFilterId, Integer> getBuildChannels()
@@ -206,9 +240,12 @@ public synchronized String toString()
206240
{
207241
return toStringHelper(this)
208242
.add("buildChannels", buildChannels)
243+
.add("filterBuildTypes", filterBuildTypes)
244+
.add("domainSizeLimitInBytes", domainSizeLimitInBytes)
209245
.add("expectedPartitionCount", expectedPartitionCount)
246+
.add("collectedPartitionCount", collectedPartitionCount)
210247
.add("collected", collected)
211-
.add("partitions", partitions)
248+
.add("summaryDomains", summaryDomains)
212249
.toString();
213250
}
214251
}

0 commit comments

Comments
 (0)