-
Notifications
You must be signed in to change notification settings - Fork 3.6k
Enable dynamic filtering in JDBC connector #8137
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
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,57 @@ | ||
| /* | ||
| * 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.plugin.jdbc; | ||
|
|
||
| import io.airlift.configuration.Config; | ||
| import io.airlift.configuration.ConfigDescription; | ||
| import io.airlift.units.Duration; | ||
| import io.airlift.units.MinDuration; | ||
|
|
||
| import javax.validation.constraints.NotNull; | ||
|
|
||
| import java.util.concurrent.TimeUnit; | ||
|
|
||
| public class DynamicFilteringJdbcConfig | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Please add a |
||
| { | ||
| private boolean enableDynamicFiltering; | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Default off? Should it be on?
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. We can turn it on in a follow-up change after running benchmarks on it. We'll also want tests with actual connectors rather than just |
||
| private Duration dynamicFilteringWaitTimeout = new Duration(0, TimeUnit.MINUTES); | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Let's use a default of 20 seconds here, that will ensure that DF collection has sufficient time to complete without hanging too long. Also, if the build side gets too big, the DF collection will automatically bail out sooner. |
||
|
|
||
| public boolean isEnableDynamicFiltering() | ||
| { | ||
| return enableDynamicFiltering; | ||
| } | ||
|
|
||
| @Config("dynamic-filtering.enabled") | ||
| public DynamicFilteringJdbcConfig setEnableDynamicFiltering(boolean enableDynamicFiltering) | ||
| { | ||
| this.enableDynamicFiltering = enableDynamicFiltering; | ||
| return this; | ||
| } | ||
|
|
||
| @NotNull | ||
| @MinDuration("0ms") | ||
| public Duration getDynamicFilteringWaitTimeout() | ||
| { | ||
| return dynamicFilteringWaitTimeout; | ||
| } | ||
|
|
||
| @Config("dynamic-filtering.wait-timeout") | ||
| @ConfigDescription("Duration to wait for completion of dynamic filters") | ||
| public DynamicFilteringJdbcConfig setDynamicFilteringWaitTimeout(Duration timeout) | ||
| { | ||
| this.dynamicFilteringWaitTimeout = timeout; | ||
| return this; | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,69 @@ | ||
| /* | ||
| * 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.plugin.jdbc; | ||
|
|
||
| import com.google.common.collect.ImmutableList; | ||
| import io.airlift.units.Duration; | ||
| import io.trino.plugin.base.session.SessionPropertiesProvider; | ||
| import io.trino.spi.connector.ConnectorSession; | ||
| import io.trino.spi.session.PropertyMetadata; | ||
|
|
||
| import javax.inject.Inject; | ||
|
|
||
| import java.util.List; | ||
|
|
||
| import static io.trino.plugin.base.session.PropertyMetadataUtil.durationProperty; | ||
| import static io.trino.spi.session.PropertyMetadata.booleanProperty; | ||
|
|
||
| public class DynamicFilteringJdbcSessionProperties | ||
| implements SessionPropertiesProvider | ||
| { | ||
| private static final String DYNAMIC_FILTERING_ENABLED = "dynamic_filtering_enabled"; | ||
| private static final String DYNAMIC_FILTERING_WAIT_TIMEOUT = "dynamic_filtering_wait_timeout"; | ||
jerryleooo marked this conversation as resolved.
Show resolved
Hide resolved
|
||
|
|
||
| private final List<PropertyMetadata<?>> properties; | ||
|
|
||
| @Inject | ||
| public DynamicFilteringJdbcSessionProperties(DynamicFilteringJdbcConfig config) | ||
| { | ||
| properties = ImmutableList.of( | ||
| booleanProperty( | ||
| DYNAMIC_FILTERING_ENABLED, | ||
| "If dynamic filtering is enabled", | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. "Enable pushdown of dynamic filters to JDBC source" |
||
| config.isEnableDynamicFiltering(), | ||
| false), | ||
| durationProperty( | ||
| DYNAMIC_FILTERING_WAIT_TIMEOUT, | ||
| "Duration to wait for completion of dynamic filters", | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. "Duration to wait for the collection of dynamic filters before starting JDBC query" |
||
| config.getDynamicFilteringWaitTimeout(), | ||
| false)); | ||
| } | ||
|
|
||
| public static boolean isEnableDynamicFiltering(ConnectorSession session) | ||
| { | ||
| return session.getProperty(DYNAMIC_FILTERING_ENABLED, Boolean.class); | ||
| } | ||
|
|
||
| public static Duration getDynamicFilteringWaitTimeout(ConnectorSession session) | ||
| { | ||
| return session.getProperty(DYNAMIC_FILTERING_WAIT_TIMEOUT, Duration.class); | ||
| } | ||
|
|
||
| @Override | ||
| public List<PropertyMetadata<?>> getSessionProperties() | ||
| { | ||
| return properties; | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -21,6 +21,7 @@ | |
| import io.trino.spi.connector.ConnectorSplit; | ||
| import io.trino.spi.connector.ConnectorTableHandle; | ||
| import io.trino.spi.connector.ConnectorTransactionHandle; | ||
| import io.trino.spi.connector.DynamicFilter; | ||
| import io.trino.spi.connector.RecordSet; | ||
|
|
||
| import javax.inject.Inject; | ||
|
|
@@ -29,7 +30,10 @@ | |
| import java.util.concurrent.ExecutorService; | ||
|
|
||
| import static com.google.common.base.Verify.verify; | ||
| import static io.trino.plugin.jdbc.DynamicFilteringJdbcSessionProperties.getDynamicFilteringWaitTimeout; | ||
| import static io.trino.plugin.jdbc.DynamicFilteringJdbcSessionProperties.isEnableDynamicFiltering; | ||
| import static java.util.Objects.requireNonNull; | ||
| import static java.util.concurrent.TimeUnit.MILLISECONDS; | ||
|
|
||
| public class JdbcRecordSetProvider | ||
| implements ConnectorRecordSetProvider | ||
|
|
@@ -44,6 +48,31 @@ public JdbcRecordSetProvider(JdbcClient jdbcClient, @ForRecordCursor ExecutorSer | |
| this.executor = requireNonNull(executor, "executor is null"); | ||
| } | ||
|
|
||
| @Override | ||
| public RecordSet getRecordSet(ConnectorTransactionHandle transaction, ConnectorSession session, ConnectorSplit split, ConnectorTableHandle table, List<? extends ColumnHandle> columns, DynamicFilter dynamicFilter) | ||
| { | ||
| if (isEnableDynamicFiltering(session)) { | ||
| long timeoutMillis = getDynamicFilteringWaitTimeout(session).toMillis(); | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I find some weird behavior when trying this Postgres as a probe side. It almost seems random whether dynamic filtering is used or not. I've set join_reordering_strategy='NONE' and also set dynamic_filtering_wait_timeout='10000ms'. Executing the same join over and over I find some sometimes it uses dynamic filters and sometimes it does not. In the debugger I see that a dynamic filter is passed here, but its tupledomain is "all". All single node between two different JDBC datasources.
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Update: This happens only with PARTITIONED joins... As expected I guess since I am testing single node. Is this waiting for something? Looks good to me. |
||
|
|
||
| // no time for further narrow down or done already | ||
| if (timeoutMillis == 0 || !dynamicFilter.isAwaitable()) { | ||
| JdbcTableHandle handle = ((JdbcTableHandle) table).withConstraint(dynamicFilter.getCurrentPredicate()); | ||
| return getRecordSet(transaction, session, split, handle, columns); | ||
| } | ||
|
|
||
| try { | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Please remove the try/catch we shouldn't be hiding any errors that result from DF pushdown |
||
| dynamicFilter.isBlocked().get(timeoutMillis, MILLISECONDS); | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This will block an execution thread in the engine and prevent other splits from running, so we need to avoid this somehow. |
||
| JdbcTableHandle handle = ((JdbcTableHandle) table).withConstraint(dynamicFilter.getCurrentPredicate()); | ||
| return getRecordSet(transaction, session, split, handle, columns); | ||
| } | ||
| catch (Exception e) { | ||
| return getRecordSet(transaction, session, split, table, columns); | ||
| } | ||
| } | ||
|
|
||
| return getRecordSet(transaction, session, split, table, columns); | ||
| } | ||
|
|
||
| @Override | ||
| public RecordSet getRecordSet(ConnectorTransactionHandle transaction, ConnectorSession session, ConnectorSplit split, ConnectorTableHandle table, List<? extends ColumnHandle> columns) | ||
| { | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -103,6 +103,16 @@ public JdbcTableHandle( | |
| this.nextSyntheticColumnId = nextSyntheticColumnId; | ||
| } | ||
|
|
||
| public JdbcTableHandle withConstraint(TupleDomain<ColumnHandle> newConstraint) | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. withConstraint -> intersectedWithConstraint |
||
| { | ||
| TupleDomain<ColumnHandle> newDomain = constraint.intersect(newConstraint); | ||
| if (newDomain == constraint) { | ||
| return this; | ||
| } | ||
|
Comment on lines
+109
to
+111
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This condition isn't really needed, we can keep it simple and remove it |
||
|
|
||
| return new JdbcTableHandle(relationHandle, newDomain, constraintExpressions, sortOrder, limit, columns, otherReferencedTables, nextSyntheticColumnId); | ||
| } | ||
|
|
||
| /** | ||
| * @deprecated Use {@code asPlainTable().getSchemaTableName()} instead, but see those methods for more information, as this is not a drop-in replacement. | ||
| */ | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This should actually pass the DynamicFIlter along.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yes, should be something like