Skip to content

Commit

Permalink
Deny creating tables with column comment if unsupported
Browse files Browse the repository at this point in the history
  • Loading branch information
ebyhr committed May 24, 2022
1 parent e96b9f4 commit 7847baa
Show file tree
Hide file tree
Showing 27 changed files with 106 additions and 21 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@
import static io.trino.plugin.accumulo.AccumuloQueryRunner.createAccumuloQueryRunner;
import static io.trino.spi.type.VarcharType.VARCHAR;
import static io.trino.testing.MaterializedResult.resultBuilder;
import static io.trino.testing.sql.TestTable.randomTableSuffix;
import static org.assertj.core.api.Assertions.assertThat;
import static org.assertj.core.api.Assertions.assertThatThrownBy;
import static org.testng.Assert.assertEquals;
Expand Down Expand Up @@ -95,6 +96,18 @@ protected TestTable createTableWithDefaultColumns()
throw new SkipException("Accumulo connector does not support column default values");
}

@Override
public void testCreateTableWithColumnComment()
{
// TODO Avoid setting hard-coded column comment
String tableName = "test_create_" + randomTableSuffix();

assertUpdate("CREATE TABLE " + tableName + " (a bigint COMMENT 'test comment')");
assertEquals(getColumnComment(tableName, "a"), "Accumulo row ID");

assertUpdate("DROP TABLE " + tableName);
}

@Override
public void testCreateTableAsSelect()
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -546,6 +546,9 @@ protected String createTableSql(RemoteTableName remoteTableName, List<String> co

protected String getColumnDefinitionSql(ConnectorSession session, ColumnMetadata column, String columnName)
{
if (column.getComment() != null) {
throw new TrinoException(NOT_SUPPORTED, "This connector does not support creating tables with column comment");
}
StringBuilder sb = new StringBuilder()
.append(quoted(columnName))
.append(" ")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -66,6 +66,7 @@ protected boolean hasBehavior(TestingConnectorBehavior connectorBehavior)
case SUPPORTS_RENAME_TABLE_ACROSS_SCHEMAS:
return false;

case SUPPORTS_CREATE_TABLE_WITH_COLUMN_COMMENT:
case SUPPORTS_COMMENT_ON_TABLE:
case SUPPORTS_COMMENT_ON_COLUMN:
case SUPPORTS_ADD_COLUMN_WITH_COMMENT:
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -79,6 +79,7 @@
import static io.trino.plugin.bigquery.BigQueryTableHandle.BigQueryPartitionType.INGESTION;
import static io.trino.plugin.bigquery.BigQueryType.toField;
import static io.trino.plugin.bigquery.BigQueryUtil.isWildcardTable;
import static io.trino.spi.StandardErrorCode.NOT_SUPPORTED;
import static java.util.Locale.ENGLISH;
import static java.util.Objects.requireNonNull;
import static java.util.function.Function.identity;
Expand Down Expand Up @@ -393,6 +394,7 @@ private void createTable(ConnectorSession session, ConnectorTableMetadata tableM
}

List<Field> fields = tableMetadata.getColumns().stream()
.peek(column -> throwIfNotNull(column.getComment()))
.map(column -> toField(column.getName(), column.getType()))
.collect(toImmutableList());

Expand Down Expand Up @@ -507,4 +509,11 @@ public RecordCursor cursor(ConnectorTransactionHandle transactionHandle, Connect
}
};
}

private static void throwIfNotNull(String comment)
{
if (comment != null) {
throw new TrinoException(NOT_SUPPORTED, "This connector does not support creating tables with column comment");
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -69,6 +69,7 @@ protected boolean hasBehavior(TestingConnectorBehavior connectorBehavior)
case SUPPORTS_RENAME_TABLE:
case SUPPORTS_NOT_NULL_CONSTRAINT:
case SUPPORTS_CREATE_TABLE_WITH_DATA:
case SUPPORTS_CREATE_TABLE_WITH_COLUMN_COMMENT:
case SUPPORTS_DELETE:
case SUPPORTS_INSERT:
case SUPPORTS_ADD_COLUMN:
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -293,6 +293,9 @@ private CassandraOutputTableHandle createTable(ConnectorTableMetadata tableMetad
ImmutableList.Builder<ExtraColumnMetadata> columnExtra = ImmutableList.builder();
columnExtra.add(new ExtraColumnMetadata(ID_COLUMN_NAME, true));
for (ColumnMetadata column : tableMetadata.getColumns()) {
if (column.getComment() != null) {
throw new TrinoException(NOT_SUPPORTED, "This connector does not support creating tables with column comment");
}
columnNames.add(column.getName());
columnTypes.add(column.getType());
columnExtra.add(new ExtraColumnMetadata(column.getName(), column.isHidden()));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -95,6 +95,9 @@ protected boolean hasBehavior(TestingConnectorBehavior connectorBehavior)
case SUPPORTS_CREATE_VIEW:
return false;

case SUPPORTS_CREATE_TABLE_WITH_COLUMN_COMMENT:
return false;

case SUPPORTS_RENAME_TABLE:
return false;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -224,15 +224,6 @@ protected TestTable createTableWithDefaultColumns()
"col_required2 Int64) ENGINE=Log");
}

@Test
public void testCreateTableWithColumnComment()
{
// TODO (https://github.com/trinodb/trino/issues/11162) Merge into BaseConnectorTest
try (TestTable table = new TestTable(getQueryRunner()::execute, "test_column_comment", "(col integer COMMENT 'column comment')")) {
assertEquals(getColumnComment(table.getName(), "col"), "column comment");
}
}

@Override
public void testCharVarcharComparison()
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -247,6 +247,9 @@ public void dropSchema(ConnectorSession session, String schemaName)
@Override
public void createTable(ConnectorSession session, ConnectorTableMetadata tableMetadata, boolean ignoreExisting)
{
if (tableMetadata.getColumns().stream().anyMatch(column -> column.getComment() != null)) {
throw new TrinoException(NOT_SUPPORTED, "This connector does not support creating tables with column comment");
}
clientSession.createTable(tableMetadata, ignoreExisting);
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -275,6 +275,22 @@ public void testCreateTable()
throw new SkipException("TODO");
}

@Override
public void testCreateTableWithColumnComment()
{
// TODO https://github.com/trinodb/trino/issues/12469 Support column comment when creating tables
String tableName = "test_create_" + randomTableSuffix();

assertQueryFails(
"CREATE TABLE " + tableName + "(" +
"id INT WITH (primary_key=true)," +
"a VARCHAR COMMENT 'test comment')" +
"WITH (partition_by_hash_columns = ARRAY['id'], partition_by_hash_buckets = 2)",
"This connector does not support creating tables with column comment");

assertUpdate("DROP TABLE IF EXISTS " + tableName);
}

@Override
public void testDropTable()
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -43,6 +43,7 @@ protected boolean hasBehavior(TestingConnectorBehavior connectorBehavior)
case SUPPORTS_AGGREGATION_PUSHDOWN_STDDEV:
case SUPPORTS_AGGREGATION_PUSHDOWN_VARIANCE:
return true;
case SUPPORTS_CREATE_TABLE_WITH_COLUMN_COMMENT:
case SUPPORTS_ADD_COLUMN_WITH_COMMENT:
case SUPPORTS_JOIN_PUSHDOWN_WITH_FULL_JOIN:
case SUPPORTS_JOIN_PUSHDOWN_WITH_DISTINCT_FROM:
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -67,6 +67,7 @@
import static com.google.common.collect.ImmutableMap.toImmutableMap;
import static io.trino.spi.StandardErrorCode.ALREADY_EXISTS;
import static io.trino.spi.StandardErrorCode.NOT_FOUND;
import static io.trino.spi.StandardErrorCode.NOT_SUPPORTED;
import static io.trino.spi.StandardErrorCode.SCHEMA_NOT_EMPTY;
import static io.trino.spi.connector.RetryMode.NO_RETRIES;
import static io.trino.spi.connector.SampleType.SYSTEM;
Expand Down Expand Up @@ -244,6 +245,9 @@ public synchronized MemoryOutputTableHandle beginCreateTable(ConnectorSession se
ImmutableList.Builder<ColumnInfo> columns = ImmutableList.builder();
for (int i = 0; i < tableMetadata.getColumns().size(); i++) {
ColumnMetadata column = tableMetadata.getColumns().get(i);
if (column.getComment() != null) {
throw new TrinoException(NOT_SUPPORTED, "This connector does not support creating tables with column comment");
}
columns.add(new ColumnInfo(new MemoryColumnHandle(i), column.getName(), column.getType()));
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -97,6 +97,7 @@ protected boolean hasBehavior(TestingConnectorBehavior connectorBehavior)
case SUPPORTS_RENAME_COLUMN:
return false;

case SUPPORTS_CREATE_TABLE_WITH_COLUMN_COMMENT:
case SUPPORTS_COMMENT_ON_TABLE:
case SUPPORTS_COMMENT_ON_COLUMN:
return false;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -135,18 +135,6 @@ public Object[][] guessFieldTypesProvider()
};
}

@Test
public void testCreateTableWithColumnComment()
{
// TODO (https://github.com/trinodb/trino/issues/11162) Merge into io.trino.testing.BaseConnectorTest#testCommentColumn
try (TestTable table = new TestTable(getQueryRunner()::execute, "test_column_comment", "(col integer COMMENT 'test')")) {
assertThat((String) computeScalar("SHOW CREATE TABLE " + table.getName()))
.isEqualTo(format("CREATE TABLE %s.%s.%s (\n" +
" col integer COMMENT 'test'\n" +
")", getSession().getCatalog().orElseThrow(), getSession().getSchema().orElseThrow(), table.getName()));
}
}

@Test
public void createTableWithEveryType()
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -63,6 +63,9 @@ protected boolean hasBehavior(TestingConnectorBehavior connectorBehavior)
case SUPPORTS_JOIN_PUSHDOWN_WITH_DISTINCT_FROM:
return false;

case SUPPORTS_CREATE_TABLE_WITH_COLUMN_COMMENT:
return false;

case SUPPORTS_COMMENT_ON_COLUMN:
case SUPPORTS_ADD_COLUMN_WITH_COMMENT:
return false;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -64,6 +64,9 @@ protected boolean hasBehavior(TestingConnectorBehavior connectorBehavior)
case SUPPORTS_COMMENT_ON_TABLE:
return false;

case SUPPORTS_CREATE_TABLE_WITH_COLUMN_COMMENT:
return false;

case SUPPORTS_ADD_COLUMN_WITH_COMMENT:
return false;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -604,6 +604,9 @@ public JdbcOutputTableHandle beginCreateTable(ConnectorSession session, Connecto
rowkeyColumn = Optional.of(ROWKEY);
}
for (ColumnMetadata column : tableColumns) {
if (column.getComment() != null) {
throw new TrinoException(NOT_SUPPORTED, "This connector does not support creating tables with column comment");
}
String columnName = getIdentifierMapping().toRemoteColumnName(connection, column.getName());
columnNames.add(columnName);
columnTypes.add(column.getType());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -71,6 +71,9 @@ protected boolean hasBehavior(TestingConnectorBehavior connectorBehavior)
case SUPPORTS_AGGREGATION_PUSHDOWN:
return false;

case SUPPORTS_CREATE_TABLE_WITH_COLUMN_COMMENT:
return false;

case SUPPORTS_COMMENT_ON_TABLE:
case SUPPORTS_COMMENT_ON_COLUMN:
case SUPPORTS_ADD_COLUMN_WITH_COMMENT:
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -596,6 +596,9 @@ public JdbcOutputTableHandle beginCreateTable(ConnectorSession session, Connecto
rowkeyColumn = Optional.of(ROWKEY);
}
for (ColumnMetadata column : tableColumns) {
if (column.getComment() != null) {
throw new TrinoException(NOT_SUPPORTED, "This connector does not support creating tables with column comment");
}
String columnName = getIdentifierMapping().toRemoteColumnName(connection, column.getName());
columnNames.add(columnName);
columnTypes.add(column.getType());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -89,6 +89,9 @@ protected boolean hasBehavior(TestingConnectorBehavior connectorBehavior)
case SUPPORTS_AGGREGATION_PUSHDOWN:
return false;

case SUPPORTS_CREATE_TABLE_WITH_COLUMN_COMMENT:
return false;

case SUPPORTS_COMMENT_ON_TABLE:
case SUPPORTS_COMMENT_ON_COLUMN:
case SUPPORTS_ADD_COLUMN_WITH_COMMENT:
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -118,6 +118,9 @@ protected boolean hasBehavior(TestingConnectorBehavior connectorBehavior)
case SUPPORTS_JOIN_PUSHDOWN:
return true;

case SUPPORTS_CREATE_TABLE_WITH_COLUMN_COMMENT:
return false;

case SUPPORTS_COMMENT_ON_TABLE:
case SUPPORTS_ADD_COLUMN_WITH_COMMENT:
return false;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -565,6 +565,9 @@ public ConnectorOutputTableHandle beginCreateTable(ConnectorSession session, Con

long columnId = 1;
for (ColumnMetadata column : tableMetadata.getColumns()) {
if (column.getComment() != null) {
throw new TrinoException(NOT_SUPPORTED, "This connector does not support creating tables with column comment");
}
columnHandles.add(new RaptorColumnHandle(column.getName(), columnId, column.getType()));
columnTypes.add(column.getType());
columnId++;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -73,6 +73,7 @@ protected boolean hasBehavior(TestingConnectorBehavior connectorBehavior)
return true;
case SUPPORTS_CREATE_SCHEMA:
case SUPPORTS_RENAME_SCHEMA:
case SUPPORTS_CREATE_TABLE_WITH_COLUMN_COMMENT:
case SUPPORTS_COMMENT_ON_TABLE:
case SUPPORTS_COMMENT_ON_COLUMN:
case SUPPORTS_ADD_COLUMN_WITH_COMMENT:
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -79,6 +79,9 @@ protected boolean hasBehavior(TestingConnectorBehavior connectorBehavior)
case SUPPORTS_JOIN_PUSHDOWN_WITH_DISTINCT_FROM:
return false;

case SUPPORTS_CREATE_TABLE_WITH_COLUMN_COMMENT:
return false;

case SUPPORTS_COMMENT_ON_TABLE:
case SUPPORTS_COMMENT_ON_COLUMN:
case SUPPORTS_ADD_COLUMN_WITH_COMMENT:
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -71,6 +71,7 @@ protected boolean hasBehavior(TestingConnectorBehavior connectorBehavior)
case SUPPORTS_JOIN_PUSHDOWN_WITH_DISTINCT_FROM:
return false;

case SUPPORTS_CREATE_TABLE_WITH_COLUMN_COMMENT:
case SUPPORTS_COMMENT_ON_TABLE:
case SUPPORTS_COMMENT_ON_COLUMN:
return false;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -82,6 +82,7 @@
import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_CREATE_MATERIALIZED_VIEW;
import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_CREATE_SCHEMA;
import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_CREATE_TABLE;
import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_CREATE_TABLE_WITH_COLUMN_COMMENT;
import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_CREATE_TABLE_WITH_DATA;
import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_CREATE_VIEW;
import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_DELETE;
Expand Down Expand Up @@ -1891,6 +1892,23 @@ public void testCreateTable()
assertFalse(getQueryRunner().tableExists(getSession(), tableNameLike));
}

@Test
public void testCreateTableWithColumnComment()
{
skipTestUnless(hasBehavior(SUPPORTS_CREATE_TABLE));

String tableName = "test_create_" + randomTableSuffix();
if (!hasBehavior(SUPPORTS_CREATE_TABLE_WITH_COLUMN_COMMENT)) {
assertQueryFails("CREATE TABLE " + tableName + " (a bigint COMMENT 'test comment')", "This connector does not support creating tables with column comment");
return;
}

assertUpdate("CREATE TABLE " + tableName + " (a bigint COMMENT 'test comment')");
assertEquals(getColumnComment(tableName, "a"), "test comment");

assertUpdate("DROP TABLE " + tableName);
}

@Test
public void testCreateTableSchemaNotFound()
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -54,6 +54,7 @@ public enum TestingConnectorBehavior

SUPPORTS_CREATE_TABLE,
SUPPORTS_CREATE_TABLE_WITH_DATA(SUPPORTS_CREATE_TABLE),
SUPPORTS_CREATE_TABLE_WITH_COLUMN_COMMENT(SUPPORTS_CREATE_TABLE),
SUPPORTS_RENAME_TABLE,
SUPPORTS_RENAME_TABLE_ACROSS_SCHEMAS(SUPPORTS_RENAME_TABLE),

Expand Down

0 comments on commit 7847baa

Please sign in to comment.