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

Multi-join pushdown appends recursively _<number> to column names so that they exceed maximum alias length #18924

Merged
Original file line number Diff line number Diff line change
Expand Up @@ -113,11 +113,17 @@ public class DefaultJdbcMetadata

private final AtomicReference<Runnable> rollbackAction = new AtomicReference<>();

public DefaultJdbcMetadata(JdbcClient jdbcClient, boolean precalculateStatisticsForPushdown, Set<JdbcQueryEventListener> jdbcQueryEventListeners)
private final SyntheticColumnHandleBuilder syntheticColumnBuilder;

public DefaultJdbcMetadata(JdbcClient jdbcClient,
boolean precalculateStatisticsForPushdown,
Comment on lines +118 to +119
Copy link
Member

Choose a reason for hiding this comment

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

We format arguments like below when putting arguments on separate lines:

    public DefaultJdbcMetadata(
            JdbcClient jdbcClient,
            boolean precalculateStatisticsForPushdown,
            Set<JdbcQueryEventListener> jdbcQueryEventListeners,
            SyntheticColumnHandleBuilder syntheticColumnBuilder)

Same for Ignite and Phoenix.

Set<JdbcQueryEventListener> jdbcQueryEventListeners,
SyntheticColumnHandleBuilder syntheticColumnBuilder)
{
this.jdbcClient = requireNonNull(jdbcClient, "jdbcClient is null");
this.precalculateStatisticsForPushdown = precalculateStatisticsForPushdown;
this.jdbcQueryEventListeners = ImmutableSet.copyOf(requireNonNull(jdbcQueryEventListeners, "queryEventListeners is null"));
this.syntheticColumnBuilder = requireNonNull(syntheticColumnBuilder, "syntheticColumnBuilder is null");
}

@Override
Expand Down Expand Up @@ -453,18 +459,14 @@ public Optional<JoinApplicationResult<ConnectorTableHandle>> applyJoin(

ImmutableMap.Builder<JdbcColumnHandle, JdbcColumnHandle> newLeftColumnsBuilder = ImmutableMap.builder();
for (JdbcColumnHandle column : jdbcClient.getColumns(session, leftHandle)) {
newLeftColumnsBuilder.put(column, JdbcColumnHandle.builderFrom(column)
.setColumnName(column.getColumnName() + "_" + nextSyntheticColumnId)
.build());
newLeftColumnsBuilder.put(column, syntheticColumnBuilder.get(column, nextSyntheticColumnId));
nextSyntheticColumnId++;
}
Map<JdbcColumnHandle, JdbcColumnHandle> newLeftColumns = newLeftColumnsBuilder.buildOrThrow();

ImmutableMap.Builder<JdbcColumnHandle, JdbcColumnHandle> newRightColumnsBuilder = ImmutableMap.builder();
for (JdbcColumnHandle column : jdbcClient.getColumns(session, rightHandle)) {
newRightColumnsBuilder.put(column, JdbcColumnHandle.builderFrom(column)
.setColumnName(column.getColumnName() + "_" + nextSyntheticColumnId)
.build());
newRightColumnsBuilder.put(column, syntheticColumnBuilder.get(column, nextSyntheticColumnId));
nextSyntheticColumnId++;
}
Map<JdbcColumnHandle, JdbcColumnHandle> newRightColumns = newRightColumnsBuilder.buildOrThrow();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,11 +28,14 @@ public class DefaultJdbcMetadataFactory
private final JdbcClient jdbcClient;
private final Set<JdbcQueryEventListener> jdbcQueryEventListeners;

protected final SyntheticColumnHandleBuilder syntheticColumnBuilder;

@Inject
public DefaultJdbcMetadataFactory(JdbcClient jdbcClient, Set<JdbcQueryEventListener> jdbcQueryEventListeners)
public DefaultJdbcMetadataFactory(JdbcClient jdbcClient, Set<JdbcQueryEventListener> jdbcQueryEventListeners, SyntheticColumnHandleBuilder syntheticColumnBuilder)
{
this.jdbcClient = requireNonNull(jdbcClient, "jdbcClient is null");
this.jdbcQueryEventListeners = ImmutableSet.copyOf(requireNonNull(jdbcQueryEventListeners, "queryEventListeners is null"));
this.syntheticColumnBuilder = requireNonNull(syntheticColumnBuilder, "syntheticColumnBuilder is null");
}

@Override
Expand All @@ -51,6 +54,6 @@ public JdbcMetadata create(JdbcTransactionHandle transaction)

protected JdbcMetadata create(JdbcClient transactionCachingJdbcClient)
{
return new DefaultJdbcMetadata(transactionCachingJdbcClient, true, jdbcQueryEventListeners);
return new DefaultJdbcMetadata(transactionCachingJdbcClient, true, jdbcQueryEventListeners, syntheticColumnBuilder);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -50,6 +50,7 @@ public void setup(Binder binder)
install(new JdbcDiagnosticModule());
install(new IdentifierMappingModule());
install(new RemoteQueryModifierModule());
install(new SyntheticColumnHandleBuilderModule());

newOptionalBinder(binder, ConnectorAccessControl.class);
newOptionalBinder(binder, QueryBuilder.class).setDefault().to(DefaultQueryBuilder.class).in(Scopes.SINGLETON);
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,39 @@
/*
* 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 static com.google.common.base.Splitter.fixedLength;
import static com.google.common.base.Verify.verify;

public class SyntheticColumnHandleBuilder
{
public static final int DEFAULT_COLUMN_ALIAS_LENGTH = 30;

public JdbcColumnHandle get(JdbcColumnHandle column, int nextSyntheticColumnId)
{
verify(nextSyntheticColumnId >= 0, "nextSyntheticColumnId rolled over and is not monotonically increasing any more");

int sequentialNumberLength = String.valueOf(nextSyntheticColumnId).length();
int originalColumnNameLength = DEFAULT_COLUMN_ALIAS_LENGTH - sequentialNumberLength - "_".length();

String columnNameTruncated = fixedLength(originalColumnNameLength)
.split(column.getColumnName())
.iterator()
.next();
String columnName = columnNameTruncated + "_" + nextSyntheticColumnId;
return JdbcColumnHandle.builderFrom(column)
.setColumnName(columnName)
.build();
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,27 @@
/*
* 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.inject.AbstractModule;
import com.google.inject.Singleton;

public class SyntheticColumnHandleBuilderModule
extends AbstractModule
{
@Override
public void configure()
{
bind(SyntheticColumnHandleBuilder.class).in(Singleton.class);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -60,19 +60,29 @@ public class TestDefaultJdbcMetadata
private DefaultJdbcMetadata metadata;
private JdbcTableHandle tableHandle;

private final SyntheticColumnHandleBuilder syntheticColumnHandleBuilder = new SyntheticColumnHandleBuilder();

@BeforeMethod
public void setUp()
throws Exception
{
database = new TestingDatabase();
metadata = new DefaultJdbcMetadata(new GroupingSetsEnabledJdbcClient(database.getJdbcClient(), Optional.empty()), false, ImmutableSet.of());
metadata = new DefaultJdbcMetadata(new GroupingSetsEnabledJdbcClient(database.getJdbcClient(),
Optional.empty()),
false,
ImmutableSet.of(),
syntheticColumnHandleBuilder);
tableHandle = metadata.getTableHandle(SESSION, new SchemaTableName("example", "numbers"));
}

@Test
public void testSupportsRetriesValidation()
{
metadata = new DefaultJdbcMetadata(new GroupingSetsEnabledJdbcClient(database.getJdbcClient(), Optional.of(false)), false, ImmutableSet.of());
metadata = new DefaultJdbcMetadata(new GroupingSetsEnabledJdbcClient(database.getJdbcClient(),
Optional.of(false)),
false,
ImmutableSet.of(),
syntheticColumnHandleBuilder);
ConnectorTableMetadata tableMetadata = new ConnectorTableMetadata(new SchemaTableName("example", "numbers"), ImmutableList.of());

assertThatThrownBy(() -> {
Expand All @@ -87,7 +97,11 @@ public void testSupportsRetriesValidation()
@Test
public void testNonTransactionalInsertValidation()
{
metadata = new DefaultJdbcMetadata(new GroupingSetsEnabledJdbcClient(database.getJdbcClient(), Optional.of(true)), false, ImmutableSet.of());
metadata = new DefaultJdbcMetadata(new GroupingSetsEnabledJdbcClient(database.getJdbcClient(),
Optional.of(true)),
false,
ImmutableSet.of(),
syntheticColumnHandleBuilder);
ConnectorTableMetadata tableMetadata = new ConnectorTableMetadata(new SchemaTableName("example", "numbers"), ImmutableList.of());

ConnectorSession session = TestingConnectorSession.builder()
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,68 @@
/*
* 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.base.VerifyException;
import org.testng.annotations.DataProvider;
import org.testng.annotations.Test;

import static io.trino.plugin.jdbc.TestingJdbcTypeHandle.JDBC_VARCHAR;
import static io.trino.spi.type.VarcharType.VARCHAR;
import static java.lang.Integer.MAX_VALUE;
import static org.assertj.core.api.Assertions.assertThat;
import static org.assertj.core.api.Assertions.assertThatThrownBy;

public class TestSyntheticColumnHandleBuilder
{
private final SyntheticColumnHandleBuilder syntheticColumnHandleBuilder = new SyntheticColumnHandleBuilder();

@DataProvider(name = "columns")
Copy link
Member

Choose a reason for hiding this comment

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

Helper method should be located after the caller.

Copy link
Member Author

Choose a reason for hiding this comment

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

Can you elaborate or provide an example. I don't think I understand your comment.

Copy link
Member

Choose a reason for hiding this comment

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

I mean replacing the method order:

    @Test(dataProvider = "columns")
    public void testColumnAliasTruncation(String columnName, int nextSynthenticId, String expectedSyntheticColumnName)
    {
        JdbcColumnHandle column = getDefaultColumnHandleBuilder()
                .setColumnName(columnName)
                .build();

        JdbcColumnHandle result = syntheticColumnHandleBuilder.get(column, nextSynthenticId);

        assertThat(result.getColumnName()).isEqualTo(expectedSyntheticColumnName);
    }

    @DataProvider(name = "columns")
    public static Object[][] testData()
    {
        return new Object[][] {
                {"column_0", 999, "column_0_999"},
                {"column_with_over_twenty_characters", 100, "column_with_over_twenty_ch_100"},
                {"column_with_over_twenty_characters", MAX_VALUE, "column_with_over_tw_2147483647"}
        };
    }

Copy link
Member

Choose a reason for hiding this comment

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

The point is that we are reading from the top to bottom. That way you read more abstract things, then if you care you go deeper and understand details. This follows Clean Code book.

public static Object[][] testData()
Copy link
Member

Choose a reason for hiding this comment

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

We usually use dataProvider suffix for @DataProvider. I would rename to columnDataProvider or something.

{
return new Object[][] {
{"column_0", 999, "column_0_999"},
{"column_with_over_twenty_characters", 100, "column_with_over_twenty_ch_100"},
{"column_with_over_twenty_characters", MAX_VALUE, "column_with_over_tw_2147483647"}
};
}

@Test(dataProvider = "columns")
public void testColumnAliasTruncation(String columnName, int nextSynthenticId, String expectedSyntheticColumnName)
{
JdbcColumnHandle column = getDefaultColumnHandleBuilder()
.setColumnName(columnName)
.build();

JdbcColumnHandle result = syntheticColumnHandleBuilder.get(column, nextSynthenticId);

assertThat(result.getColumnName()).isEqualTo(expectedSyntheticColumnName);
}

@Test
public void testNegativeSyntheticId()
{
JdbcColumnHandle column = getDefaultColumnHandleBuilder()
.setColumnName("column_0")
.build();

assertThatThrownBy(() -> syntheticColumnHandleBuilder.get(column, -2147483648)).isInstanceOf(VerifyException.class);
Copy link
Member

Choose a reason for hiding this comment

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

It would better to verify the error message.

Copy link
Member Author

Choose a reason for hiding this comment

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

Likewise here. Can you explain in more detail?

Copy link
Member

Choose a reason for hiding this comment

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

        assertThatThrownBy(() -> syntheticColumnHandleBuilder.get(column, -2147483648))
                .isInstanceOf(VerifyException.class)
                .hasMessage("nextSyntheticColumnId rolled over and is not monotonically increasing any more");

}

private static JdbcColumnHandle.Builder getDefaultColumnHandleBuilder()
{
return JdbcColumnHandle.builder()
.setJdbcTypeHandle(JDBC_VARCHAR)
.setColumnType(VARCHAR);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@
import io.trino.plugin.jdbc.JdbcClient;
import io.trino.plugin.jdbc.JdbcMetadata;
import io.trino.plugin.jdbc.JdbcQueryEventListener;
import io.trino.plugin.jdbc.SyntheticColumnHandleBuilder;

import java.util.Set;

Expand All @@ -30,15 +31,17 @@ public class IgniteJdbcMetadataFactory
private final Set<JdbcQueryEventListener> jdbcQueryEventListeners;

@Inject
public IgniteJdbcMetadataFactory(JdbcClient jdbcClient, Set<JdbcQueryEventListener> jdbcQueryEventListeners)
public IgniteJdbcMetadataFactory(JdbcClient jdbcClient,
Set<JdbcQueryEventListener> jdbcQueryEventListeners,
SyntheticColumnHandleBuilder syntheticColumnHandleBuilder)
{
super(jdbcClient, jdbcQueryEventListeners);
super(jdbcClient, jdbcQueryEventListeners, syntheticColumnHandleBuilder);
this.jdbcQueryEventListeners = ImmutableSet.copyOf(requireNonNull(jdbcQueryEventListeners, "jdbcQueryEventListeners is null"));
}

@Override
protected JdbcMetadata create(JdbcClient transactionCachingJdbcClient)
{
return new IgniteMetadata(transactionCachingJdbcClient, jdbcQueryEventListeners);
return new IgniteMetadata(transactionCachingJdbcClient, jdbcQueryEventListeners, syntheticColumnBuilder);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@
import io.trino.plugin.jdbc.JdbcTableHandle;
import io.trino.plugin.jdbc.JdbcTypeHandle;
import io.trino.plugin.jdbc.RemoteTableName;
import io.trino.plugin.jdbc.SyntheticColumnHandleBuilder;
import io.trino.spi.TrinoException;
import io.trino.spi.connector.ColumnHandle;
import io.trino.spi.connector.ColumnMetadata;
Expand Down Expand Up @@ -57,9 +58,11 @@ public class IgniteMetadata
private final JdbcClient igniteClient;

@Inject
public IgniteMetadata(JdbcClient igniteClient, Set<JdbcQueryEventListener> jdbcQueryEventListeners)
public IgniteMetadata(JdbcClient igniteClient,
Set<JdbcQueryEventListener> jdbcQueryEventListeners,
SyntheticColumnHandleBuilder syntheticColumnHandleBuilder)
{
super(igniteClient, false, jdbcQueryEventListeners);
super(igniteClient, false, jdbcQueryEventListeners, syntheticColumnHandleBuilder);
this.igniteClient = requireNonNull(igniteClient, "igniteClient is null");
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,19 +17,24 @@
import io.airlift.testing.Closeables;
import io.trino.testing.QueryRunner;
import io.trino.testing.sql.SqlExecutor;
import io.trino.testing.sql.TestTable;
import org.testng.annotations.AfterClass;
import org.testng.annotations.Test;

import static io.trino.plugin.jdbc.SyntheticColumnHandleBuilder.DEFAULT_COLUMN_ALIAS_LENGTH;
import static io.trino.plugin.oracle.TestingOracleServer.TEST_PASS;
import static io.trino.plugin.oracle.TestingOracleServer.TEST_SCHEMA;
import static io.trino.plugin.oracle.TestingOracleServer.TEST_USER;
import static java.lang.String.format;
import static java.util.stream.Collectors.joining;
import static java.util.stream.IntStream.range;
import static org.assertj.core.api.Assertions.assertThat;

public class TestOracleConnectorTest
extends BaseOracleConnectorTest
{
private static final String MAXIMUM_LENGTH_COLUMN_IDENTIFIER = "z".repeat(DEFAULT_COLUMN_ALIAS_LENGTH);

private TestingOracleServer oracleServer;

@Override
Expand Down Expand Up @@ -83,4 +88,16 @@ protected SqlExecutor onRemoteDatabase()
{
return oracleServer::execute;
}

@Test
public void testPushdownJoinWithLongNameSucceeds()
Copy link
Member

Choose a reason for hiding this comment

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

Can we have a test with multiple super long column names used in the same Trino query that are almost the same (zzz..zza, zzz..zzb, ...)? Such columns should represent different data. The thing I was thinking we could check is to validate that synthetic names do not generate conflicts.

Or is it an overkill?

Copy link
Member

Choose a reason for hiding this comment

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

yes, see the tests being added in #18984

fortunately the remote databases which silently truncate long identifiers (SQLServer, Postgres, Redshift) all prevent this bug from happening because they realise that the truncated alias is now ambiguous.

{
try (TestTable table = new TestTable(getQueryRunner()::execute, "long_identifier", "(%s bigint)".formatted(MAXIMUM_LENGTH_COLUMN_IDENTIFIER))) {
assertThat(query(joinPushdownEnabled(getSession()), """
SELECT r.name, t.%s, n.name
FROM %s t JOIN region r ON r.regionkey = t.%s
JOIN nation n ON r.regionkey = n.regionkey""".formatted(MAXIMUM_LENGTH_COLUMN_IDENTIFIER, table.getName(), MAXIMUM_LENGTH_COLUMN_IDENTIFIER)))
.isFullyPushedDown();
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -50,6 +50,7 @@
import io.trino.plugin.jdbc.QueryBuilder;
import io.trino.plugin.jdbc.ReusableConnectionFactoryModule;
import io.trino.plugin.jdbc.StatsCollecting;
import io.trino.plugin.jdbc.SyntheticColumnHandleBuilderModule;
import io.trino.plugin.jdbc.TypeHandlingJdbcConfig;
import io.trino.plugin.jdbc.TypeHandlingJdbcSessionProperties;
import io.trino.plugin.jdbc.credential.EmptyCredentialProvider;
Expand Down Expand Up @@ -150,6 +151,7 @@ protected void setup(Binder binder)
install(new JdbcDiagnosticModule());
install(new IdentifierMappingModule());
install(new DecimalModule());
install(new SyntheticColumnHandleBuilderModule());
}

private void checkConfiguration(String connectionUrl)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@
import io.trino.plugin.jdbc.JdbcTableHandle;
import io.trino.plugin.jdbc.JdbcTypeHandle;
import io.trino.plugin.jdbc.RemoteTableName;
import io.trino.plugin.jdbc.SyntheticColumnHandleBuilder;
import io.trino.spi.TrinoException;
import io.trino.spi.connector.AggregateFunction;
import io.trino.spi.connector.AggregationApplicationResult;
Expand Down Expand Up @@ -84,9 +85,12 @@ public class PhoenixMetadata
private final IdentifierMapping identifierMapping;

@Inject
public PhoenixMetadata(PhoenixClient phoenixClient, IdentifierMapping identifierMapping, Set<JdbcQueryEventListener> jdbcQueryEventListeners)
public PhoenixMetadata(PhoenixClient phoenixClient,
IdentifierMapping identifierMapping,
Set<JdbcQueryEventListener> jdbcQueryEventListeners,
SyntheticColumnHandleBuilder syntheticColumnHandleBuilder)
{
super(phoenixClient, false, jdbcQueryEventListeners);
super(phoenixClient, false, jdbcQueryEventListeners, syntheticColumnHandleBuilder);
this.phoenixClient = requireNonNull(phoenixClient, "phoenixClient is null");
this.identifierMapping = requireNonNull(identifierMapping, "identifierMapping is null");
}
Expand Down