Skip to content

Commit

Permalink
Provide more information to connectors to control aggregation pushdown
Browse files Browse the repository at this point in the history
  • Loading branch information
hashhar committed Aug 2, 2021
1 parent 480a621 commit c0bb821
Show file tree
Hide file tree
Showing 7 changed files with 10 additions and 10 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -170,9 +170,9 @@ public WriteMapping toWriteMapping(ConnectorSession session, Type type)
}

@Override
public boolean supportsAggregationPushdown(ConnectorSession session, JdbcTableHandle table, List<List<ColumnHandle>> groupingSets)
public boolean supportsAggregationPushdown(ConnectorSession session, JdbcTableHandle table, List<AggregateFunction> aggregates, Map<String, ColumnHandle> assignments, List<List<ColumnHandle>> groupingSets)
{
return delegate.supportsAggregationPushdown(session, table, groupingSets);
return delegate.supportsAggregationPushdown(session, table, aggregates, assignments, groupingSets);
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -248,7 +248,7 @@ public Optional<AggregationApplicationResult<ConnectorTableHandle>> applyAggrega
// Global aggregation is represented by [[]]
verify(!groupingSets.isEmpty(), "No grouping sets provided");

if (!jdbcClient.supportsAggregationPushdown(session, handle, groupingSets)) {
if (!jdbcClient.supportsAggregationPushdown(session, handle, aggregates, assignments, groupingSets)) {
// JDBC client implementation prevents pushdown for the given table
return Optional.empty();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -107,9 +107,9 @@ public WriteMapping toWriteMapping(ConnectorSession session, Type type)
}

@Override
public boolean supportsAggregationPushdown(ConnectorSession session, JdbcTableHandle table, List<List<ColumnHandle>> groupingSets)
public boolean supportsAggregationPushdown(ConnectorSession session, JdbcTableHandle table, List<AggregateFunction> aggregates, Map<String, ColumnHandle> assignments, List<List<ColumnHandle>> groupingSets)
{
return delegate().supportsAggregationPushdown(session, table, groupingSets);
return delegate().supportsAggregationPushdown(session, table, aggregates, assignments, groupingSets);
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -64,7 +64,7 @@ default boolean schemaExists(ConnectorSession session, String schema)

WriteMapping toWriteMapping(ConnectorSession session, Type type);

default boolean supportsAggregationPushdown(ConnectorSession session, JdbcTableHandle table, List<List<ColumnHandle>> groupingSets)
default boolean supportsAggregationPushdown(ConnectorSession session, JdbcTableHandle table, List<AggregateFunction> aggregates, Map<String, ColumnHandle> assignments, List<List<ColumnHandle>> groupingSets)
{
return true;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -127,9 +127,9 @@ public WriteMapping toWriteMapping(ConnectorSession session, Type type)
}

@Override
public boolean supportsAggregationPushdown(ConnectorSession session, JdbcTableHandle table, List<List<ColumnHandle>> groupingSets)
public boolean supportsAggregationPushdown(ConnectorSession session, JdbcTableHandle table, List<AggregateFunction> aggregates, Map<String, ColumnHandle> assignments, List<List<ColumnHandle>> groupingSets)
{
return delegate().supportsAggregationPushdown(session, table, groupingSets);
return delegate().supportsAggregationPushdown(session, table, aggregates, assignments, groupingSets);
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -403,7 +403,7 @@ protected JdbcClient delegate()
}

@Override
public boolean supportsAggregationPushdown(ConnectorSession session, JdbcTableHandle table, List<List<ColumnHandle>> groupingSets)
public boolean supportsAggregationPushdown(ConnectorSession session, JdbcTableHandle table, List<AggregateFunction> aggregates, Map<String, ColumnHandle> assignments, List<List<ColumnHandle>> groupingSets)
{
// disable aggregation pushdown for any table named no_agg_pushdown
return !"no_aggregation_pushdown".equalsIgnoreCase(table.getRequiredNamedRelation().getRemoteTableName().getTableName());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -76,7 +76,7 @@ public TestingH2JdbcClient(BaseJdbcConfig config, ConnectionFactory connectionFa
}

@Override
public boolean supportsAggregationPushdown(ConnectorSession session, JdbcTableHandle table, List<List<ColumnHandle>> groupingSets)
public boolean supportsAggregationPushdown(ConnectorSession session, JdbcTableHandle table, List<AggregateFunction> aggregates, Map<String, ColumnHandle> assignments, List<List<ColumnHandle>> groupingSets)
{
// GROUP BY with GROUPING SETS is not supported
return groupingSets.size() == 1;
Expand Down

0 comments on commit c0bb821

Please sign in to comment.