diff --git a/core/trino-main/pom.xml b/core/trino-main/pom.xml
index dd69613bff5b..3b2f57ace22f 100644
--- a/core/trino-main/pom.xml
+++ b/core/trino-main/pom.xml
@@ -344,6 +344,11 @@
pcollections
+
+ org.roaringbitmap
+ RoaringBitmap
+
+
org.weakref
jmxutils
diff --git a/core/trino-main/src/main/java/io/trino/execution/DynamicFilterConfig.java b/core/trino-main/src/main/java/io/trino/execution/DynamicFilterConfig.java
index 10c85e062cfa..15fec226b288 100644
--- a/core/trino-main/src/main/java/io/trino/execution/DynamicFilterConfig.java
+++ b/core/trino-main/src/main/java/io/trino/execution/DynamicFilterConfig.java
@@ -21,6 +21,7 @@
import io.airlift.units.MaxDataSize;
import javax.validation.constraints.Min;
+import javax.validation.constraints.NotNull;
import static io.airlift.units.DataSize.Unit.KILOBYTE;
import static io.airlift.units.DataSize.Unit.MEGABYTE;
@@ -31,14 +32,14 @@
"dynamic-filtering-max-per-driver-size",
"experimental.dynamic-filtering-max-per-driver-size",
"dynamic-filtering-range-row-limit-per-driver",
- "experimental.dynamic-filtering-refresh-interval"
+ "experimental.dynamic-filtering-refresh-interval",
+ "dynamic-filtering.service-thread-count"
})
public class DynamicFilterConfig
{
private boolean enableDynamicFiltering = true;
private boolean enableCoordinatorDynamicFiltersDistribution = true;
private boolean enableLargeDynamicFilters;
- private int serviceThreadCount = 2;
private int smallBroadcastMaxDistinctValuesPerDriver = 200;
private DataSize smallBroadcastMaxSizePerDriver = DataSize.of(20, KILOBYTE);
@@ -48,6 +49,7 @@ public class DynamicFilterConfig
private DataSize smallPartitionedMaxSizePerDriver = DataSize.of(10, KILOBYTE);
private int smallPartitionedRangeRowLimitPerDriver = 100;
private DataSize smallPartitionedMaxSizePerOperator = DataSize.of(100, KILOBYTE);
+ private DataSize smallMaxSizePerFilter = DataSize.of(1, MEGABYTE);
private int largeBroadcastMaxDistinctValuesPerDriver = 5_000;
private DataSize largeBroadcastMaxSizePerDriver = DataSize.of(500, KILOBYTE);
@@ -57,6 +59,7 @@ public class DynamicFilterConfig
private DataSize largePartitionedMaxSizePerDriver = DataSize.of(50, KILOBYTE);
private int largePartitionedRangeRowLimitPerDriver = 1_000;
private DataSize largePartitionedMaxSizePerOperator = DataSize.of(500, KILOBYTE);
+ private DataSize largeMaxSizePerFilter = DataSize.of(5, MEGABYTE);
public boolean isEnableDynamicFiltering()
{
@@ -96,19 +99,6 @@ public DynamicFilterConfig setEnableLargeDynamicFilters(boolean enableLargeDynam
return this;
}
- @Min(1)
- public int getServiceThreadCount()
- {
- return serviceThreadCount;
- }
-
- @Config("dynamic-filtering.service-thread-count")
- public DynamicFilterConfig setServiceThreadCount(int serviceThreadCount)
- {
- this.serviceThreadCount = serviceThreadCount;
- return this;
- }
-
@Min(0)
public int getSmallBroadcastMaxDistinctValuesPerDriver()
{
@@ -213,6 +203,20 @@ public DynamicFilterConfig setSmallPartitionedMaxSizePerOperator(DataSize smallP
return this;
}
+ @NotNull
+ @MaxDataSize("10MB")
+ public DataSize getSmallMaxSizePerFilter()
+ {
+ return smallMaxSizePerFilter;
+ }
+
+ @Config("dynamic-filtering.small.max-size-per-filter")
+ public DynamicFilterConfig setSmallMaxSizePerFilter(DataSize smallMaxSizePerFilter)
+ {
+ this.smallMaxSizePerFilter = smallMaxSizePerFilter;
+ return this;
+ }
+
@Min(0)
public int getLargeBroadcastMaxDistinctValuesPerDriver()
{
@@ -316,4 +320,18 @@ public DynamicFilterConfig setLargePartitionedMaxSizePerOperator(DataSize largeP
this.largePartitionedMaxSizePerOperator = largePartitionedMaxSizePerOperator;
return this;
}
+
+ @NotNull
+ @MaxDataSize("10MB")
+ public DataSize getLargeMaxSizePerFilter()
+ {
+ return largeMaxSizePerFilter;
+ }
+
+ @Config("dynamic-filtering.large.max-size-per-filter")
+ public DynamicFilterConfig setLargeMaxSizePerFilter(DataSize largeMaxSizePerFilter)
+ {
+ this.largeMaxSizePerFilter = largeMaxSizePerFilter;
+ return this;
+ }
}
diff --git a/core/trino-main/src/main/java/io/trino/server/DynamicFilterService.java b/core/trino-main/src/main/java/io/trino/server/DynamicFilterService.java
index c696d07a461b..6060253963ae 100644
--- a/core/trino-main/src/main/java/io/trino/server/DynamicFilterService.java
+++ b/core/trino-main/src/main/java/io/trino/server/DynamicFilterService.java
@@ -16,15 +16,14 @@
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.annotations.VisibleForTesting;
-import com.google.common.collect.HashMultimap;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Multimap;
-import com.google.common.collect.SetMultimap;
import com.google.common.util.concurrent.ListenableFuture;
import com.google.common.util.concurrent.SettableFuture;
import com.google.inject.Inject;
+import io.airlift.units.DataSize;
import io.airlift.units.Duration;
import io.trino.Session;
import io.trino.execution.DynamicFilterConfig;
@@ -52,29 +51,31 @@
import io.trino.sql.planner.plan.JoinNode;
import io.trino.sql.planner.plan.PlanNode;
import io.trino.sql.planner.plan.SemiJoinNode;
+import org.roaringbitmap.RoaringBitmap;
-import javax.annotation.PreDestroy;
import javax.annotation.concurrent.GuardedBy;
import javax.annotation.concurrent.ThreadSafe;
import java.util.ArrayList;
import java.util.Collection;
-import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Optional;
-import java.util.OptionalInt;
+import java.util.Queue;
import java.util.Set;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.atomic.AtomicReference;
import java.util.function.Consumer;
import static com.google.common.base.Functions.identity;
import static com.google.common.base.MoreObjects.toStringHelper;
+import static com.google.common.base.Preconditions.checkArgument;
import static com.google.common.base.Preconditions.checkState;
+import static com.google.common.base.Verify.verify;
import static com.google.common.collect.ImmutableList.toImmutableList;
import static com.google.common.collect.ImmutableMap.toImmutableMap;
import static com.google.common.collect.ImmutableSet.toImmutableSet;
@@ -82,11 +83,13 @@
import static com.google.common.collect.Sets.intersection;
import static com.google.common.collect.Sets.newConcurrentHashSet;
import static com.google.common.collect.Sets.union;
+import static com.google.common.util.concurrent.MoreExecutors.directExecutor;
+import static io.airlift.concurrent.MoreFutures.addSuccessCallback;
+import static io.airlift.concurrent.MoreFutures.getFutureValue;
import static io.airlift.concurrent.MoreFutures.toCompletableFuture;
import static io.airlift.concurrent.MoreFutures.unmodifiableFuture;
import static io.airlift.concurrent.MoreFutures.whenAnyComplete;
-import static io.airlift.concurrent.Threads.daemonThreadsNamed;
-import static io.airlift.units.Duration.succinctNanos;
+import static io.trino.SystemSessionProperties.isEnableLargeDynamicFilters;
import static io.trino.spi.connector.DynamicFilter.EMPTY;
import static io.trino.spi.predicate.Domain.union;
import static io.trino.sql.DynamicFilters.extractDynamicFilters;
@@ -96,7 +99,6 @@
import static io.trino.sql.planner.SystemPartitioningHandle.SOURCE_DISTRIBUTION;
import static java.lang.String.format;
import static java.util.Objects.requireNonNull;
-import static java.util.concurrent.Executors.newFixedThreadPool;
@ThreadSafe
public class DynamicFilterService
@@ -104,32 +106,16 @@ public class DynamicFilterService
private final Metadata metadata;
private final FunctionManager functionManager;
private final TypeOperators typeOperators;
- private final ExecutorService executor;
+ private final DynamicFilterConfig dynamicFilterConfig;
private final Map dynamicFilterContexts = new ConcurrentHashMap<>();
@Inject
public DynamicFilterService(Metadata metadata, FunctionManager functionManager, TypeOperators typeOperators, DynamicFilterConfig dynamicFilterConfig)
- {
- this(
- metadata,
- functionManager,
- typeOperators,
- newFixedThreadPool(dynamicFilterConfig.getServiceThreadCount(), daemonThreadsNamed("DynamicFilterService")));
- }
-
- @VisibleForTesting
- public DynamicFilterService(Metadata metadata, FunctionManager functionManager, TypeOperators typeOperators, ExecutorService executor)
{
this.metadata = requireNonNull(metadata, "metadata is null");
this.functionManager = requireNonNull(functionManager, "functionManager is null");
this.typeOperators = requireNonNull(typeOperators, "typeOperators is null");
- this.executor = requireNonNull(executor, "executor is null");
- }
-
- @PreDestroy
- public void stop()
- {
- executor.shutdownNow();
+ this.dynamicFilterConfig = requireNonNull(dynamicFilterConfig, "dynamicFilterConfig is null");
}
public void registerQuery(SqlQueryExecution sqlQueryExecution, SubPlan fragmentedPlan)
@@ -166,9 +152,18 @@ public void registerQuery(
dynamicFilters,
lazyDynamicFilters,
replicatedDynamicFilters,
+ getDynamicFilterSizeLimit(session),
0));
}
+ private DataSize getDynamicFilterSizeLimit(Session session)
+ {
+ if (isEnableLargeDynamicFilters(session)) {
+ return dynamicFilterConfig.getLargeMaxSizePerFilter();
+ }
+ return dynamicFilterConfig.getSmallMaxSizePerFilter();
+ }
+
public void registerQueryRetry(QueryId queryId, int attemptId)
{
DynamicFilterContext context = dynamicFilterContexts.get(queryId);
@@ -395,7 +390,6 @@ public void addTaskDynamicFilters(TaskId taskId, Map ne
taskId,
taskAttemptId);
context.addTaskDynamicFilters(taskId, newDynamicFilters);
- executor.submit(() -> collectDynamicFilters(taskId.getStageId(), Optional.of(newDynamicFilters.keySet())));
}
public void stageCannotScheduleMoreTasks(StageId stageId, int attemptId, int numberOfTasks)
@@ -412,7 +406,6 @@ public void stageCannotScheduleMoreTasks(StageId stageId, int attemptId, int num
stageId,
attemptId);
context.stageCannotScheduleMoreTasks(stageId, numberOfTasks);
- executor.submit(() -> collectDynamicFilters(stageId, Optional.empty()));
}
public static Set getOutboundDynamicFilters(PlanFragment plan)
@@ -423,38 +416,6 @@ public static Set getOutboundDynamicFilters(PlanFragment plan)
getProducedDynamicFilters(plan.getRoot())));
}
- private void collectDynamicFilters(StageId stageId, Optional> selectedFilters)
- {
- DynamicFilterContext context = dynamicFilterContexts.get(stageId.getQueryId());
- if (context == null) {
- // query has been removed
- return;
- }
-
- OptionalInt stageNumberOfTasks = context.getNumberOfTasks(stageId);
- Map> newDynamicFilters = context.getTaskDynamicFilters(stageId, selectedFilters).entrySet().stream()
- .filter(stageDomains -> {
- if (stageDomains.getValue().stream().anyMatch(Domain::isAll)) {
- // if one of the domains is all, we don't need to get dynamic filters from all tasks
- return true;
- }
-
- if (!stageDomains.getValue().isEmpty() && context.getReplicatedDynamicFilters().contains(stageDomains.getKey())) {
- // for replicated dynamic filters it's enough to get dynamic filter from a single task
- checkState(
- stageDomains.getValue().size() == 1,
- "Replicated dynamic filter should be collected from single task");
- return true;
- }
-
- // check if all tasks of a dynamic filter source have reported dynamic filter summary
- return stageNumberOfTasks.isPresent() && stageDomains.getValue().size() == stageNumberOfTasks.getAsInt();
- })
- .collect(toImmutableMap(Map.Entry::getKey, Map.Entry::getValue));
-
- context.addDynamicFilters(newDynamicFilters);
- }
-
@VisibleForTesting
Optional getSummary(QueryId queryId, DynamicFilterId filterId)
{
@@ -469,7 +430,6 @@ private TupleDomain translateSummaryToTupleDomain(
TypeProvider typeProvider)
{
Collection descriptors = descriptorMultimap.get(filterId);
- checkState(descriptors != null, "No descriptors for dynamic filter %s", filterId);
Domain summary = dynamicFilterContext.getDynamicFilterSummaries().get(filterId);
return TupleDomain.withColumnDomains(descriptors.stream()
.collect(toImmutableMap(
@@ -702,37 +662,247 @@ public String toString()
}
}
- /*
- * DynamicFilterContext can be fully lock-free since computing dynamic filter summaries
- * is idempotent. Concurrent computations of DF summaries should produce exact same result
- * when partial (from tasks) DFs are available. Partial DFs are only removed when
- * final dynamic filter summary is computed.
- */
+ private static class DynamicFilterCollectionContext
+ {
+ private final boolean replicated;
+ private final long domainSizeLimitInBytes;
+ @GuardedBy("collectedTasks")
+ private final RoaringBitmap collectedTasks = new RoaringBitmap();
+ private final Queue summaryDomains = new ConcurrentLinkedQueue<>();
+ private final AtomicLong summaryDomainsRetainedSizeInBytes = new AtomicLong();
+
+ @GuardedBy("this")
+ private volatile Integer expectedTaskCount;
+ @GuardedBy("this")
+ private int collectedTaskCount;
+
+ private final long start = System.nanoTime();
+ private final AtomicReference collectionDuration = new AtomicReference<>();
+ @GuardedBy("this")
+ private volatile boolean collected;
+ private final SettableFuture collectedDomainsFuture = SettableFuture.create();
+
+ private DynamicFilterCollectionContext(boolean replicated, long domainSizeLimitInBytes)
+ {
+ this.replicated = replicated;
+ this.domainSizeLimitInBytes = domainSizeLimitInBytes;
+ }
+
+ public void collect(TaskId taskId, Domain domain)
+ {
+ if (collected) {
+ return;
+ }
+
+ if (replicated) {
+ collectReplicated(domain);
+ }
+ else {
+ collectPartitioned(taskId, domain);
+ }
+ }
+
+ private void collectReplicated(Domain domain)
+ {
+ if (domain.getRetainedSizeInBytes() > domainSizeLimitInBytes) {
+ domain = domain.simplify(1);
+ }
+ if (domain.getRetainedSizeInBytes() > domainSizeLimitInBytes) {
+ domain = Domain.all(domain.getType());
+ }
+ Domain result;
+ synchronized (this) {
+ if (collected) {
+ return;
+ }
+ collectedTaskCount++;
+ collected = true;
+ result = domain;
+ }
+ collectionDuration.set(Duration.succinctNanos(System.nanoTime() - start));
+ collectedDomainsFuture.set(result);
+ }
+
+ private void collectPartitioned(TaskId taskId, Domain domain)
+ {
+ synchronized (collectedTasks) {
+ if (!collectedTasks.checkedAdd(taskId.getPartitionId())) {
+ return;
+ }
+ }
+
+ summaryDomainsRetainedSizeInBytes.addAndGet(domain.getRetainedSizeInBytes());
+ summaryDomains.add(domain);
+ unionSummaryDomainsIfNecessary(false);
+
+ Domain result;
+ synchronized (this) {
+ if (collected) {
+ clearSummaryDomains();
+ return;
+ }
+ collectedTaskCount++;
+ boolean allPartitionsCollected = expectedTaskCount != null && expectedTaskCount == collectedTaskCount;
+ if (allPartitionsCollected) {
+ // run final compaction as previous concurrent compactions may have left more than a single domain
+ unionSummaryDomainsIfNecessary(true);
+ }
+
+ boolean sizeLimitExceeded = false;
+ Domain allDomain = null;
+ Domain summary = summaryDomains.poll();
+ // summary can be null as another concurrent summary compaction may be running
+ if (summary != null) {
+ long originalSize = summary.getRetainedSizeInBytes();
+ if (summary.getRetainedSizeInBytes() > domainSizeLimitInBytes) {
+ summary = summary.simplify(1);
+ }
+ if (summary.getRetainedSizeInBytes() > domainSizeLimitInBytes) {
+ sizeLimitExceeded = true;
+ allDomain = Domain.all(summary.getType());
+ summaryDomainsRetainedSizeInBytes.addAndGet(-originalSize);
+ }
+ else {
+ summaryDomainsRetainedSizeInBytes.addAndGet(summary.getRetainedSizeInBytes() - originalSize);
+ summaryDomains.add(summary);
+ }
+ }
+
+ boolean collectionFinished = sizeLimitExceeded || domain.isAll() || allPartitionsCollected;
+ if (!collectionFinished) {
+ return;
+ }
+ collected = true;
+ if (sizeLimitExceeded) {
+ result = allDomain;
+ }
+ else if (domain.isAll()) {
+ clearSummaryDomains();
+ result = domain;
+ }
+ else {
+ verify(allPartitionsCollected, "allPartitionsCollected is expected to be true");
+ int summaryDomainsCount = summaryDomains.size();
+ verify(summaryDomainsCount == 1, "summaryDomainsCount is expected to be equal to 1, got: %s", summaryDomainsCount);
+ result = summaryDomains.poll();
+ verify(result != null);
+ long currentSize = summaryDomainsRetainedSizeInBytes.addAndGet(-result.getRetainedSizeInBytes());
+ verify(currentSize == 0, "currentSize is expected to be zero: %s", currentSize);
+ }
+ }
+
+ collectionDuration.set(Duration.succinctNanos(System.nanoTime() - start));
+ collectedDomainsFuture.set(result);
+ }
+
+ private void unionSummaryDomainsIfNecessary(boolean force)
+ {
+ if (summaryDomainsRetainedSizeInBytes.get() < domainSizeLimitInBytes && !force) {
+ return;
+ }
+
+ List domains = new ArrayList<>();
+ long domainsRetainedSizeInBytes = 0;
+ while (true) {
+ Domain domain = summaryDomains.poll();
+ if (domain == null) {
+ break;
+ }
+ domains.add(domain);
+ domainsRetainedSizeInBytes += domain.getRetainedSizeInBytes();
+ }
+
+ if (domains.isEmpty()) {
+ return;
+ }
+
+ Domain union = union(domains);
+ summaryDomainsRetainedSizeInBytes.addAndGet(union.getRetainedSizeInBytes() - domainsRetainedSizeInBytes);
+ long currentSize = summaryDomainsRetainedSizeInBytes.get();
+ verify(currentSize >= 0, "currentSize is expected to be greater than or equal to zero: %s", currentSize);
+ summaryDomains.add(union);
+ }
+
+ private void clearSummaryDomains()
+ {
+ long domainsRetainedSizeInBytes = 0;
+ while (true) {
+ Domain domain = summaryDomains.poll();
+ if (domain == null) {
+ break;
+ }
+ domainsRetainedSizeInBytes += domain.getRetainedSizeInBytes();
+ }
+ summaryDomainsRetainedSizeInBytes.addAndGet(-domainsRetainedSizeInBytes);
+ long currentSize = summaryDomainsRetainedSizeInBytes.get();
+ verify(currentSize >= 0, "currentSize is expected to be greater than or equal to zero: %s", currentSize);
+ }
+
+ public void setExpectedTaskCount(int count)
+ {
+ if (collected || expectedTaskCount != null) {
+ return;
+ }
+ checkArgument(count > 0, "count is expected to be greater than zero: %s", count);
+
+ Domain result;
+ synchronized (this) {
+ if (collected || expectedTaskCount != null) {
+ return;
+ }
+ expectedTaskCount = count;
+ verify(collectedTaskCount <= expectedTaskCount,
+ "collectedTaskCount is expected to be less than or equal to %s, got: %s",
+ expectedTaskCount,
+ collectedTaskCount);
+ if (collectedTaskCount != expectedTaskCount) {
+ return;
+ }
+ // run union one more time
+ unionSummaryDomainsIfNecessary(true);
+
+ verify(summaryDomains.size() == 1);
+ result = summaryDomains.poll();
+ verify(result != null);
+ long currentSize = summaryDomainsRetainedSizeInBytes.addAndGet(-result.getRetainedSizeInBytes());
+ verify(currentSize == 0, "currentSize is expected to be zero: %s", currentSize);
+ }
+
+ collectionDuration.set(Duration.succinctNanos(System.nanoTime() - start));
+ collectedDomainsFuture.set(result);
+ }
+
+ public ListenableFuture getCollectedDomainFuture()
+ {
+ return collectedDomainsFuture;
+ }
+
+ public Optional getCollectionDuration()
+ {
+ return Optional.ofNullable(collectionDuration.get());
+ }
+ }
+
private static class DynamicFilterContext
{
private final Session session;
- private final Map dynamicFilterSummaries = new ConcurrentHashMap<>();
- private final Map dynamicFilterCollectionTime = new ConcurrentHashMap<>();
private final Set dynamicFilters;
- private final Map> lazyDynamicFilters;
private final Set replicatedDynamicFilters;
+ private final DataSize dynamicFilterSizeLimit;
+ private final Map> lazyDynamicFilters;
+ private final Map dynamicFilterCollectionContexts;
+
private final Map> stageDynamicFilters = new ConcurrentHashMap<>();
private final Map stageNumberOfTasks = new ConcurrentHashMap<>();
- // when map value for given filter id is empty it means that dynamic filter has already been collected
- // and no partial task domains are required
- private final Map> taskDynamicFilters = new ConcurrentHashMap<>();
- @GuardedBy("dynamicFilterConsumers")
- // This should not be a ConcurrentHashMap because we want to prevent concurrent addition of new consumers during the
- // removal of existing consumers from this map in addDynamicFilters. This ensures that new consumers don't miss filter completion.
- private final Map>>> dynamicFilterConsumers = new HashMap<>();
+
private final int attemptId;
- private final long queryAttemptStartTime = System.nanoTime();
private DynamicFilterContext(
Session session,
Set dynamicFilters,
Set lazyDynamicFilters,
Set replicatedDynamicFilters,
+ DataSize dynamicFilterSizeLimit,
int attemptId)
{
this.session = requireNonNull(session, "session is null");
@@ -741,10 +911,17 @@ private DynamicFilterContext(
this.lazyDynamicFilters = lazyDynamicFilters.stream()
.collect(toImmutableMap(identity(), filter -> SettableFuture.create()));
this.replicatedDynamicFilters = requireNonNull(replicatedDynamicFilters, "replicatedDynamicFilters is null");
- dynamicFilters.forEach(filter -> {
- taskDynamicFilters.put(filter, new ConcurrentHashMap<>());
- dynamicFilterConsumers.put(filter, new ArrayList<>());
- });
+ this.dynamicFilterSizeLimit = requireNonNull(dynamicFilterSizeLimit, "dynamicFilterSizeLimit is null");
+ ImmutableMap.Builder collectionContexts = ImmutableMap.builder();
+ for (DynamicFilterId dynamicFilterId : dynamicFilters) {
+ DynamicFilterCollectionContext collectionContext = new DynamicFilterCollectionContext(replicatedDynamicFilters.contains(dynamicFilterId), dynamicFilterSizeLimit.toBytes());
+ collectionContexts.put(dynamicFilterId, collectionContext);
+ SettableFuture lazyDynamicFilterFuture = this.lazyDynamicFilters.get(dynamicFilterId);
+ if (lazyDynamicFilterFuture != null) {
+ collectionContext.getCollectedDomainFuture().addListener(() -> lazyDynamicFilterFuture.set(null), directExecutor());
+ }
+ }
+ dynamicFilterCollectionContexts = collectionContexts.buildOrThrow();
this.attemptId = attemptId;
}
@@ -755,27 +932,16 @@ DynamicFilterContext createContextForQueryRetry(int attemptId)
dynamicFilters,
lazyDynamicFilters.keySet(),
replicatedDynamicFilters,
+ dynamicFilterSizeLimit,
attemptId);
}
void addDynamicFilterConsumer(Set dynamicFilterIds, Consumer
+
+ org.roaringbitmap
+ RoaringBitmap
+ 0.9.25
+
+
org.sonatype.aether
aether-api