Skip to content

Commit

Permalink
Block creating long table names in JDBC and MongoDB connectors
Browse files Browse the repository at this point in the history
Previously, PostgreSQL connector silently truncated
table names to 63 characters.
Let MongoDB creates a collection explicitly because
implicit creation doesn't happen in case of long identifiers.
  • Loading branch information
ebyhr committed Jul 4, 2022
1 parent 31de900 commit d54a5fb
Show file tree
Hide file tree
Showing 22 changed files with 309 additions and 1 deletion.
Expand Up @@ -555,6 +555,8 @@ protected JdbcOutputTableHandle createTable(ConnectorSession session, ConnectorT
String remoteTargetTableName = identifierMapping.toRemoteTableName(identity, connection, remoteSchema, targetTableName);
String catalog = connection.getCatalog();

verifyTableName(connection.getMetaData(), remoteTargetTableName);

List<ColumnMetadata> columns = tableMetadata.getColumns();
ImmutableList.Builder<String> columnNames = ImmutableList.builderWithExpectedSize(columns.size());
ImmutableList.Builder<Type> columnTypes = ImmutableList.builderWithExpectedSize(columns.size());
Expand Down Expand Up @@ -693,6 +695,7 @@ protected void renameTable(ConnectorSession session, String catalogName, String
try (Connection connection = connectionFactory.openConnection(session)) {
String newSchemaName = newTable.getSchemaName();
String newTableName = newTable.getTableName();
verifyTableName(connection.getMetaData(), newTableName);
ConnectorIdentity identity = session.getIdentity();
String newRemoteSchemaName = identifierMapping.toRemoteSchemaName(identity, connection, newSchemaName);
String newRemoteTableName = identifierMapping.toRemoteTableName(identity, connection, newRemoteSchemaName, newTableName);
Expand Down Expand Up @@ -1075,6 +1078,12 @@ public void truncateTable(ConnectorSession session, JdbcTableHandle handle)
execute(session, sql);
}

protected void verifyTableName(DatabaseMetaData databaseMetadata, String tableName)
throws SQLException
{
// expect remote databases throw an exception for unsupported table names
}

protected String quoted(@Nullable String catalog, @Nullable String schema, String table)
{
StringBuilder sb = new StringBuilder();
Expand Down
Expand Up @@ -28,6 +28,7 @@
import org.testng.annotations.Test;

import java.util.List;
import java.util.OptionalInt;

import static com.google.common.base.Strings.nullToEmpty;
import static io.trino.plugin.bigquery.BigQueryQueryRunner.BigQuerySqlExecutor;
Expand Down Expand Up @@ -708,6 +709,18 @@ public void testMissingWildcardTable()
.hasMessageEndingWith("does not match any table.");
}

@Override
protected OptionalInt maxTableNameLength()
{
return OptionalInt.of(1024);
}

@Override
protected void verifyTableNameLengthFailurePermissible(Throwable e)
{
assertThat(e).hasMessageContaining("Invalid table ID");
}

private void onBigQuery(@Language("SQL") String sql)
{
bigQuerySqlExecutor.execute(sql);
Expand Down
Expand Up @@ -39,6 +39,7 @@
import java.time.format.DateTimeFormatter;
import java.util.List;
import java.util.Optional;
import java.util.OptionalInt;

import static com.datastax.oss.driver.api.core.data.ByteUtils.toHexString;
import static com.google.common.io.BaseEncoding.base16;
Expand Down Expand Up @@ -1321,6 +1322,18 @@ public void testRowLevelDelete()
.hasStackTraceContaining("Delete without primary key or partition key is not supported");
}

@Override
protected OptionalInt maxTableNameLength()
{
return OptionalInt.of(48);
}

@Override
protected void verifyTableNameLengthFailurePermissible(Throwable e)
{
assertThat(e).hasMessageContaining("Table names shouldn't be more than 48 characters long");
}

private void assertSelect(String tableName, boolean createdByTrino)
{
Type inetType = createdByTrino ? createUnboundedVarcharType() : createVarcharType(45);
Expand Down
Expand Up @@ -674,6 +674,25 @@ protected TestTable simpleTable()
return new TestTable(onRemoteDatabase(), "tpch.simple_table", "(col BIGINT) Engine=Log", ImmutableList.of("1", "2"));
}

@Override
public void testCreateTableWithLongTableName()
{
// Override because ClickHouse connector can create a table which can't be dropped
String baseTableName = "test_create_" + randomTableSuffix();
String validTableName = baseTableName + "z".repeat(maxTableNameLength().orElseThrow() - baseTableName.length());

assertUpdate("CREATE TABLE " + validTableName + " (a bigint)");
assertTrue(getQueryRunner().tableExists(getSession(), validTableName));
assertThatThrownBy(() -> assertUpdate("DROP TABLE " + validTableName))
.hasMessageMatching("(?s).*(Bad path syntax|File name too long).*");

String invalidTableName = baseTableName + "z".repeat(maxTableNameLength().orElseThrow() - baseTableName.length() + 1);
assertThatThrownBy(() -> query("CREATE TABLE " + invalidTableName + " (a bigint)"))
.hasMessageMatching("(?s).*(Cannot open file|File name too long).*");
// ClickHouse lefts a table even if the above statement failed
assertTrue(getQueryRunner().tableExists(getSession(), validTableName));
}

@Override
protected SqlExecutor onRemoteDatabase()
{
Expand Down
Expand Up @@ -16,6 +16,8 @@
import com.google.common.collect.ImmutableMap;
import io.trino.testing.QueryRunner;

import java.util.OptionalInt;

import static io.trino.plugin.clickhouse.ClickHouseQueryRunner.createClickHouseQueryRunner;
import static org.assertj.core.api.Assertions.assertThatThrownBy;

Expand Down Expand Up @@ -58,4 +60,11 @@ public void testCreateTableAsSelectWithTableComment()
assertThatThrownBy(super::testCreateTableAsSelectWithTableComment)
.hasMessageMatching("(?s).* Syntax error: .* COMMENT 'test comment'.*");
}

@Override
protected OptionalInt maxTableNameLength()
{
// The numeric value depends on file system
return OptionalInt.of(255 - ".sql.tmp".length());
}
}
Expand Up @@ -16,6 +16,8 @@
import com.google.common.collect.ImmutableMap;
import io.trino.testing.QueryRunner;

import java.util.OptionalInt;

import static io.trino.plugin.clickhouse.ClickHouseQueryRunner.createClickHouseQueryRunner;
import static io.trino.plugin.clickhouse.TestingClickHouseServer.CLICKHOUSE_LATEST_IMAGE;

Expand All @@ -35,4 +37,11 @@ protected QueryRunner createQueryRunner()
.buildOrThrow(),
REQUIRED_TPCH_TABLES);
}

@Override
protected OptionalInt maxTableNameLength()
{
// The numeric value depends on file system
return OptionalInt.of(255 - ".sql.detached".length());
}
}
Expand Up @@ -36,6 +36,7 @@

import java.util.List;
import java.util.Optional;
import java.util.OptionalInt;
import java.util.Set;

import static com.google.common.collect.ImmutableList.toImmutableList;
Expand Down Expand Up @@ -499,6 +500,18 @@ protected String createSchemaSql(String schemaName)
return "CREATE SCHEMA " + schemaName + " WITH (location = 's3://" + bucketName + "/" + schemaName + "')";
}

@Override
protected OptionalInt maxTableNameLength()
{
return OptionalInt.of(128);
}

@Override
protected void verifyTableNameLengthFailurePermissible(Throwable e)
{
assertThat(e).hasMessageMatching("(?s)(.*Read timed out)|(.*\"`TBL_NAME`\" that has maximum length of 128.*)");
}

private Set<String> getActiveFiles(String tableName)
{
return getActiveFiles(tableName, getQueryRunner().getDefaultSession());
Expand Down
Expand Up @@ -81,6 +81,7 @@
import java.util.Locale;
import java.util.Map;
import java.util.Optional;
import java.util.OptionalInt;
import java.util.Set;
import java.util.StringJoiner;
import java.util.function.BiConsumer;
Expand Down Expand Up @@ -8326,6 +8327,19 @@ protected TestTable createTableWithDefaultColumns()
throw new SkipException("Hive connector does not support column default values");
}

@Override
protected OptionalInt maxTableNameLength()
{
// This value depends on metastore type
return OptionalInt.of(255);
}

@Override
protected void verifyTableNameLengthFailurePermissible(Throwable e)
{
assertThat(e).hasMessageContaining("Failed to create directory");
}

private Session withTimestampPrecision(Session session, HiveTimestampPrecision precision)
{
return Session.builder(session)
Expand Down
Expand Up @@ -69,6 +69,7 @@
import java.util.Map;
import java.util.NoSuchElementException;
import java.util.Optional;
import java.util.OptionalInt;
import java.util.Set;
import java.util.concurrent.TimeUnit;
import java.util.function.Consumer;
Expand Down Expand Up @@ -4398,6 +4399,19 @@ public void testSelectWithMoreThanOneSnapshotOfTheSameTable()
assertUpdate(format("DROP TABLE %s", tableName));
}

@Override
protected OptionalInt maxTableNameLength()
{
// This value depends on metastore type
return OptionalInt.of(255);
}

@Override
protected void verifyTableNameLengthFailurePermissible(Throwable e)
{
assertThat(e).hasMessageContaining("Failed to create file");
}

private Session prepareCleanUpSession()
{
return Session.builder(getSession())
Expand Down
Expand Up @@ -372,6 +372,29 @@ public void testCreateTable()
//assertFalse(getQueryRunner().tableExists(getSession(), tableNameLike));
}

@Override
public void testCreateTableWithLongTableName()
{
// Overridden because DDL in base class can't create Kudu table due to lack of primary key and required table properties
String baseTableName = "test_create_" + randomTableSuffix();
String validTableName = baseTableName + "z".repeat(256 - baseTableName.length());

assertUpdate("CREATE TABLE " + validTableName + "(" +
"id INT WITH (primary_key=true)," +
"a VARCHAR)" +
"WITH (partition_by_hash_columns = ARRAY['id'], partition_by_hash_buckets = 2)");
assertTrue(getQueryRunner().tableExists(getSession(), validTableName));
assertUpdate("DROP TABLE " + validTableName);

String invalidTableName = baseTableName + "z".repeat(256 - baseTableName.length() + 1);
assertThatThrownBy(() -> query("CREATE TABLE " + invalidTableName + "(" +
"id INT WITH (primary_key=true)," +
"a VARCHAR)" +
"WITH (partition_by_hash_columns = ARRAY['id'], partition_by_hash_buckets = 2)"))
.hasMessageContaining("invalid table name");
assertFalse(getQueryRunner().tableExists(getSession(), validTableName));
}

@Override
public void testCreateTableWithColumnComment()
{
Expand Down
Expand Up @@ -21,6 +21,7 @@
import org.testng.annotations.Test;

import java.util.Optional;
import java.util.OptionalInt;

import static com.google.common.base.Strings.nullToEmpty;
import static io.trino.spi.type.VarcharType.VARCHAR;
Expand Down Expand Up @@ -302,4 +303,16 @@ protected String errorMessageForInsertIntoNotNullColumn(String columnName)
{
return format("Failed to insert data: .* \\(conn=.*\\) Field '%s' doesn't have a default value", columnName);
}

@Override
protected OptionalInt maxTableNameLength()
{
return OptionalInt.of(64);
}

@Override
protected void verifyTableNameLengthFailurePermissible(Throwable e)
{
assertThat(e).hasMessageContaining("Incorrect table name");
}
}
Expand Up @@ -206,7 +206,7 @@ public void createTable(SchemaTableName name, List<MongoColumnHandle> columns, O
throw new SchemaNotFoundException(name.getSchemaName());
}
createTableMetadata(name, columns, comment);
// collection is created implicitly
client.getDatabase(name.getSchemaName()).createCollection(name.getTableName());
}

public void dropTable(SchemaTableName tableName)
Expand Down
Expand Up @@ -37,6 +37,7 @@
import java.util.Arrays;
import java.util.Date;
import java.util.Optional;
import java.util.OptionalInt;

import static java.lang.String.format;
import static java.nio.charset.StandardCharsets.UTF_8;
Expand Down Expand Up @@ -576,6 +577,18 @@ public void testAddColumnConcurrently()
throw new SkipException("TODO");
}

@Override
protected OptionalInt maxTableNameLength()
{
return OptionalInt.of(120 - "tpch.".length());
}

@Override
protected void verifyTableNameLengthFailurePermissible(Throwable e)
{
assertThat(e).hasMessageMatching(".*fully qualified namespace .* is too long.*");
}

private void assertOneNotNullResult(String query)
{
MaterializedResult results = getQueryRunner().execute(getSession(), query).toTestTypes();
Expand Down
Expand Up @@ -24,6 +24,7 @@
import org.testng.annotations.Test;

import java.util.Optional;
import java.util.OptionalInt;

import static com.google.common.base.Strings.nullToEmpty;
import static com.google.common.collect.Iterables.getOnlyElement;
Expand Down Expand Up @@ -382,6 +383,18 @@ private String getLongInClause(int start, int length)
return "orderkey IN (" + longValues + ")";
}

@Override
protected OptionalInt maxTableNameLength()
{
return OptionalInt.of(64);
}

@Override
protected void verifyTableNameLengthFailurePermissible(Throwable e)
{
assertThat(e).hasMessageMatching("Identifier name .* is too long");
}

@Override
protected SqlExecutor onRemoteDatabase()
{
Expand Down
Expand Up @@ -27,6 +27,7 @@

import java.util.List;
import java.util.Optional;
import java.util.OptionalInt;

import static io.trino.plugin.oracle.TestingOracleServer.TEST_USER;
import static io.trino.spi.type.VarcharType.VARCHAR;
Expand Down Expand Up @@ -475,6 +476,18 @@ protected void verifyConcurrentAddColumnFailurePermissible(Exception e)
.hasMessage("ORA-14411: The DDL cannot be run concurrently with other DDLs\n");
}

@Override
protected OptionalInt maxTableNameLength()
{
return OptionalInt.of(30);
}

@Override
protected void verifyTableNameLengthFailurePermissible(Throwable e)
{
assertThat(e).hasMessage("ORA-00972: identifier is too long\n");
}

private void predicatePushdownTest(String oracleType, String oracleLiteral, String operator, String filterLiteral)
{
String tableName = ("test_pdown_" + oracleType.replaceAll("[^a-zA-Z0-9]", ""))
Expand Down

0 comments on commit d54a5fb

Please sign in to comment.