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

Add toggle for specifying dynamic filtering refresh interval #4388

Merged
merged 1 commit into from
Jul 27, 2020
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 @@ -22,13 +22,13 @@
import io.airlift.units.Duration;
import io.prestosql.execution.SqlQueryExecution;
import io.prestosql.execution.StageState;
import io.prestosql.execution.TaskManagerConfig;
import io.prestosql.spi.QueryId;
import io.prestosql.spi.connector.ColumnHandle;
import io.prestosql.spi.connector.DynamicFilter;
import io.prestosql.spi.predicate.Domain;
import io.prestosql.spi.predicate.TupleDomain;
import io.prestosql.sql.DynamicFilters;
import io.prestosql.sql.analyzer.FeaturesConfig;
import io.prestosql.sql.planner.Symbol;
import io.prestosql.sql.planner.optimizations.PlanNodeSearcher;
import io.prestosql.sql.planner.plan.DynamicFilterId;
Expand Down Expand Up @@ -73,7 +73,7 @@
@ThreadSafe
public class DynamicFilterService
{
private final Duration statusRefreshMaxWait;
private final Duration dynamicFilteringRefreshInterval;
private final ScheduledExecutorService collectDynamicFiltersExecutor = newSingleThreadScheduledExecutor(daemonThreadsNamed("DynamicFilterService"));

@GuardedBy("this") // for updates
Expand All @@ -86,15 +86,15 @@ public class DynamicFilterService
private final Map<QueryId, Set<DynamicFilterId>> queryReplicatedDynamicFilters = new ConcurrentHashMap<>();

@Inject
public DynamicFilterService(TaskManagerConfig taskConfig)
public DynamicFilterService(FeaturesConfig featuresConfig)
{
this.statusRefreshMaxWait = requireNonNull(taskConfig, "taskConfig is null").getStatusRefreshMaxWait();
this.dynamicFilteringRefreshInterval = requireNonNull(featuresConfig, "featuresConfig is null").getDynamicFilteringRefreshInterval();
}

@PostConstruct
public void start()
{
collectDynamicFiltersExecutor.scheduleWithFixedDelay(this::collectDynamicFilters, 0, statusRefreshMaxWait.toMillis(), MILLISECONDS);
collectDynamicFiltersExecutor.scheduleWithFixedDelay(this::collectDynamicFilters, 0, dynamicFilteringRefreshInterval.toMillis(), MILLISECONDS);
}

@PreDestroy
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,8 @@
import io.airlift.units.DataSize;
import io.airlift.units.Duration;
import io.airlift.units.MaxDataSize;
import io.airlift.units.MaxDuration;
import io.airlift.units.MinDuration;
import io.prestosql.operator.aggregation.arrayagg.ArrayAggGroupImplementation;
import io.prestosql.operator.aggregation.histogram.HistogramGroupImplementation;
import io.prestosql.operator.aggregation.multimapagg.MultimapAggGroupImplementation;
Expand All @@ -41,6 +43,7 @@
import static io.airlift.units.DataSize.Unit.MEGABYTE;
import static io.prestosql.sql.analyzer.RegexLibrary.JONI;
import static java.util.Objects.requireNonNull;
import static java.util.concurrent.TimeUnit.MILLISECONDS;
import static java.util.concurrent.TimeUnit.MINUTES;

@DefunctConfig({
Expand Down Expand Up @@ -132,6 +135,7 @@ public class FeaturesConfig
private boolean enableDynamicFiltering = true;
private int dynamicFilteringMaxPerDriverRowCount = 100;
private DataSize dynamicFilteringMaxPerDriverSize = DataSize.of(10, KILOBYTE);
private Duration dynamicFilteringRefreshInterval = new Duration(200, MILLISECONDS);

private DataSize filterAndProjectMinOutputPageSize = DataSize.of(500, KILOBYTE);
private int filterAndProjectMinOutputPageRowCount = 256;
Expand Down Expand Up @@ -787,6 +791,21 @@ public FeaturesConfig setDynamicFilteringMaxPerDriverSize(DataSize dynamicFilter
return this;
}

@MinDuration("1ms")
@MaxDuration("10s")
Copy link
Member

Choose a reason for hiding this comment

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

Nit: testcase for validation?

@NotNull
public Duration getDynamicFilteringRefreshInterval()
{
return dynamicFilteringRefreshInterval;
}

@Config("experimental.dynamic-filtering-refresh-interval")
public FeaturesConfig setDynamicFilteringRefreshInterval(Duration dynamicFilteringRefreshInterval)
{
this.dynamicFilteringRefreshInterval = dynamicFilteringRefreshInterval;
return this;
}

public boolean isOptimizeMixedDistinctAggregations()
{
return optimizeMixedDistinctAggregations;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,14 +19,14 @@
import io.prestosql.execution.StageId;
import io.prestosql.execution.StageState;
import io.prestosql.execution.TaskId;
import io.prestosql.execution.TaskManagerConfig;
import io.prestosql.server.DynamicFilterService.StageDynamicFilters;
import io.prestosql.spi.QueryId;
import io.prestosql.spi.connector.DynamicFilter;
import io.prestosql.spi.connector.TestingColumnHandle;
import io.prestosql.spi.predicate.Domain;
import io.prestosql.spi.predicate.TupleDomain;
import io.prestosql.sql.DynamicFilters;
import io.prestosql.sql.analyzer.FeaturesConfig;
import io.prestosql.sql.planner.Symbol;
import io.prestosql.sql.planner.plan.DynamicFilterId;
import io.prestosql.sql.tree.Expression;
Expand Down Expand Up @@ -56,7 +56,7 @@ public class TestDynamicFilterService
@Test
public void testDynamicFilterSummaryCompletion()
{
DynamicFilterService dynamicFilterService = new DynamicFilterService(new TaskManagerConfig());
DynamicFilterService dynamicFilterService = new DynamicFilterService(new FeaturesConfig());
DynamicFilterId filterId = new DynamicFilterId("df");
QueryId queryId = new QueryId("query");
StageId stageId = new StageId(queryId, 0);
Expand Down Expand Up @@ -101,7 +101,7 @@ public void testDynamicFilterSummaryCompletion()
@Test
public void testDynamicFilter()
{
DynamicFilterService dynamicFilterService = new DynamicFilterService(new TaskManagerConfig());
DynamicFilterService dynamicFilterService = new DynamicFilterService(new FeaturesConfig());
DynamicFilterId filterId1 = new DynamicFilterId("df1");
DynamicFilterId filterId2 = new DynamicFilterId("df2");
DynamicFilterId filterId3 = new DynamicFilterId("df3");
Expand Down Expand Up @@ -257,7 +257,7 @@ public void testDynamicFilter()
@Test
public void testReplicatedDynamicFilter()
{
DynamicFilterService dynamicFilterService = new DynamicFilterService(new TaskManagerConfig());
DynamicFilterService dynamicFilterService = new DynamicFilterService(new FeaturesConfig());
DynamicFilterId filterId1 = new DynamicFilterId("df1");
Expression df1 = expression("DF_SYMBOL1");
QueryId queryId = new QueryId("query");
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,7 @@
import static io.prestosql.sql.analyzer.FeaturesConfig.JoinReorderingStrategy.NONE;
import static io.prestosql.sql.analyzer.RegexLibrary.JONI;
import static io.prestosql.sql.analyzer.RegexLibrary.RE2J;
import static java.util.concurrent.TimeUnit.MILLISECONDS;
import static java.util.concurrent.TimeUnit.MINUTES;
import static java.util.concurrent.TimeUnit.SECONDS;

Expand Down Expand Up @@ -112,6 +113,7 @@ public void testDefaults()
.setEnableDynamicFiltering(true)
.setDynamicFilteringMaxPerDriverRowCount(100)
.setDynamicFilteringMaxPerDriverSize(DataSize.of(10, KILOBYTE))
.setDynamicFilteringRefreshInterval(new Duration(200, MILLISECONDS))
.setIgnoreDownstreamPreferences(false)
.setOmitDateTimeTypePrecision(false)
.setIterativeRuleBasedColumnPruning(true));
Expand Down Expand Up @@ -188,6 +190,7 @@ public void testExplicitPropertyMappings()
.put("enable-dynamic-filtering", "false")
.put("dynamic-filtering-max-per-driver-row-count", "256")
.put("dynamic-filtering-max-per-driver-size", "64kB")
.put("experimental.dynamic-filtering-refresh-interval", "300ms")
.put("optimizer.ignore-downstream-preferences", "true")
.put("deprecated.omit-datetime-type-precision", "true")
.put("optimizer.iterative-rule-based-column-pruning", "false")
Expand Down Expand Up @@ -261,6 +264,7 @@ public void testExplicitPropertyMappings()
.setEnableDynamicFiltering(false)
.setDynamicFilteringMaxPerDriverRowCount(256)
.setDynamicFilteringMaxPerDriverSize(DataSize.of(64, KILOBYTE))
.setDynamicFilteringRefreshInterval(new Duration(300, MILLISECONDS))
.setIgnoreDownstreamPreferences(true)
.setOmitDateTimeTypePrecision(true)
.setIterativeRuleBasedColumnPruning(false);
Expand Down