Skip to content

Commit

Permalink
Deny creating tables with table comment if unsupported
Browse files Browse the repository at this point in the history
  • Loading branch information
ebyhr committed May 27, 2022
1 parent 67e5478 commit bb0d814
Show file tree
Hide file tree
Showing 26 changed files with 113 additions and 0 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -102,6 +102,9 @@ public ConnectorOutputTableHandle beginCreateTable(ConnectorSession session, Con
if (retryMode != NO_RETRIES) {
throw new TrinoException(NOT_SUPPORTED, "This connector does not support query retries");
}
if (tableMetadata.getComment().isPresent()) {
throw new TrinoException(NOT_SUPPORTED, "This connector does not support creating tables with table comment");
}

checkNoRollback();

Expand Down Expand Up @@ -136,6 +139,9 @@ private void rollbackCreateTable(AccumuloTable table)
@Override
public void createTable(ConnectorSession session, ConnectorTableMetadata tableMetadata, boolean ignoreExisting)
{
if (tableMetadata.getComment().isPresent()) {
throw new TrinoException(NOT_SUPPORTED, "This connector does not support creating tables with table comment");
}
client.createTable(tableMetadata);
}

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

case SUPPORTS_CREATE_TABLE_WITH_TABLE_COMMENT:
return false;

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 @@ -540,6 +540,9 @@ protected JdbcOutputTableHandle createTable(ConnectorSession session, ConnectorT

protected String createTableSql(RemoteTableName remoteTableName, List<String> columns, ConnectorTableMetadata tableMetadata)
{
if (tableMetadata.getComment().isPresent()) {
throw new TrinoException(NOT_SUPPORTED, "This connector does not support creating tables with table comment");
}
checkArgument(tableMetadata.getProperties().isEmpty(), "Unsupported table properties: %s", tableMetadata.getProperties());
return format("CREATE TABLE %s (%s)", quoted(remoteTableName), join(", ", columns));
}
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_TABLE_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 @@ -371,6 +372,9 @@ public void dropSchema(ConnectorSession session, String schemaName)
@Override
public void createTable(ConnectorSession session, ConnectorTableMetadata tableMetadata, boolean ignoreExisting)
{
if (tableMetadata.getComment().isPresent()) {
throw new TrinoException(NOT_SUPPORTED, "This connector does not support creating tables with table comment");
}
try {
createTable(session, tableMetadata);
}
Expand Down
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_TABLE_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 @@ -288,6 +288,9 @@ public ConnectorOutputTableHandle beginCreateTable(ConnectorSession session, Con

private CassandraOutputTableHandle createTable(ConnectorTableMetadata tableMetadata)
{
if (tableMetadata.getComment().isPresent()) {
throw new TrinoException(NOT_SUPPORTED, "This connector does not support creating tables with table comment");
}
ImmutableList.Builder<String> columnNames = ImmutableList.builder();
ImmutableList.Builder<Type> columnTypes = ImmutableList.builder();
ImmutableList.Builder<ExtraColumnMetadata> columnExtra = ImmutableList.builder();
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_TABLE_COMMENT:
return false;

case SUPPORTS_RENAME_TABLE:
return false;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -42,4 +42,20 @@ public void testCommentTable()
assertThatThrownBy(super::testCommentTable)
.hasMessageContaining("Code: 62, e.displayText() = DB::Exception: Syntax error");
}

@Override
public void testCreateTableWithTableComment()
{
// Table comment is unsupported in old ClickHouse version
assertThatThrownBy(super::testCreateTableWithTableComment)
.hasMessageMatching("(?s).* Syntax error: .* COMMENT 'test comment'.*");
}

@Override
public void testCreateTableAsSelectWithTableComment()
{
// Table comment is unsupported in old ClickHouse version
assertThatThrownBy(super::testCreateTableAsSelectWithTableComment)
.hasMessageMatching("(?s).* Syntax error: .* COMMENT 'test comment'.*");
}
}
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.getComment().isPresent()) {
throw new TrinoException(NOT_SUPPORTED, "This connector does not support creating tables with table comment");
}
clientSession.createTable(tableMetadata, ignoreExisting);
}

Expand Down Expand Up @@ -329,6 +332,9 @@ public ConnectorOutputTableHandle beginCreateTable(
if (retryMode != NO_RETRIES) {
throw new TrinoException(NOT_SUPPORTED, "This connector does not support query retries");
}
if (tableMetadata.getComment().isPresent()) {
throw new TrinoException(NOT_SUPPORTED, "This connector does not support creating tables with table comment");
}
PartitionDesign design = KuduTableProperties.getPartitionDesign(tableMetadata.getProperties());
boolean generateUUID = !design.hasPartitions();
ConnectorTableMetadata finalTableMetadata = tableMetadata;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -70,6 +70,7 @@ protected boolean hasBehavior(TestingConnectorBehavior connectorBehavior)
case SUPPORTS_DELETE:
return true;
case SUPPORTS_RENAME_SCHEMA:
case SUPPORTS_CREATE_TABLE_WITH_TABLE_COMMENT:
case SUPPORTS_COMMENT_ON_TABLE:
case SUPPORTS_COMMENT_ON_COLUMN:
case SUPPORTS_ARRAY:
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 @@ -235,6 +236,9 @@ public synchronized void createTable(ConnectorSession session, ConnectorTableMet
@Override
public synchronized MemoryOutputTableHandle beginCreateTable(ConnectorSession session, ConnectorTableMetadata tableMetadata, Optional<ConnectorTableLayout> layout, RetryMode retryMode)
{
if (tableMetadata.getComment().isPresent()) {
throw new TrinoException(NOT_SUPPORTED, "This connector does not support creating tables with table comment");
}
checkSchemaExists(tableMetadata.getTable().getSchemaName());
checkTableNotExists(tableMetadata.getTable());
long tableId = nextTableId.getAndIncrement();
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_TABLE_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 @@ -61,6 +61,9 @@ protected boolean hasBehavior(TestingConnectorBehavior connectorBehavior)
case SUPPORTS_JOIN_PUSHDOWN_WITH_DISTINCT_FROM:
return false;

case SUPPORTS_CREATE_TABLE_WITH_TABLE_COMMENT:
return false;

case SUPPORTS_COMMENT_ON_TABLE:
return false;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -572,6 +572,9 @@ public Optional<String> getTableComment(ResultSet resultSet)
@Override
public JdbcOutputTableHandle beginCreateTable(ConnectorSession session, ConnectorTableMetadata tableMetadata)
{
if (tableMetadata.getComment().isPresent()) {
throw new TrinoException(NOT_SUPPORTED, "This connector does not support creating tables with table comment");
}
SchemaTableName schemaTableName = tableMetadata.getTable();
String schema = schemaTableName.getSchemaName();
String table = schemaTableName.getTableName();
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_AGGREGATION_PUSHDOWN:
return false;

case SUPPORTS_CREATE_TABLE_WITH_TABLE_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 @@ -564,6 +564,9 @@ public Optional<String> getTableComment(ResultSet resultSet)
@Override
public JdbcOutputTableHandle beginCreateTable(ConnectorSession session, ConnectorTableMetadata tableMetadata)
{
if (tableMetadata.getComment().isPresent()) {
throw new TrinoException(NOT_SUPPORTED, "This connector does not support creating tables with table comment");
}
SchemaTableName schemaTableName = tableMetadata.getTable();
String schema = schemaTableName.getSchemaName();
String table = schemaTableName.getTableName();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -89,6 +89,7 @@ protected boolean hasBehavior(TestingConnectorBehavior connectorBehavior)
case SUPPORTS_AGGREGATION_PUSHDOWN:
return false;

case SUPPORTS_CREATE_TABLE_WITH_TABLE_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 @@ -118,6 +118,7 @@ protected boolean hasBehavior(TestingConnectorBehavior connectorBehavior)
case SUPPORTS_JOIN_PUSHDOWN:
return true;

case SUPPORTS_CREATE_TABLE_WITH_TABLE_COMMENT:
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 @@ -550,6 +550,9 @@ public ConnectorOutputTableHandle beginCreateTable(ConnectorSession session, Con
if (retryMode != NO_RETRIES) {
throw new TrinoException(NOT_SUPPORTED, "This connector does not support query retries");
}
if (tableMetadata.getComment().isPresent()) {
throw new TrinoException(NOT_SUPPORTED, "This connector does not support creating tables with table comment");
}

if (viewExists(session, tableMetadata.getTable())) {
throw new TrinoException(ALREADY_EXISTS, "View already exists: " + tableMetadata.getTable());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -74,6 +74,7 @@ protected boolean hasBehavior(TestingConnectorBehavior connectorBehavior)
return true;
case SUPPORTS_CREATE_SCHEMA:
case SUPPORTS_RENAME_SCHEMA:
case SUPPORTS_CREATE_TABLE_WITH_TABLE_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 @@ protected boolean hasBehavior(TestingConnectorBehavior connectorBehavior)
case SUPPORTS_JOIN_PUSHDOWN_WITH_DISTINCT_FROM:
return false;

case SUPPORTS_CREATE_TABLE_WITH_TABLE_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 @@ -858,6 +858,9 @@ protected boolean isSupportedJoinCondition(ConnectorSession session, JdbcJoinCon
@Override
protected String createTableSql(RemoteTableName remoteTableName, List<String> columns, ConnectorTableMetadata tableMetadata)
{
if (tableMetadata.getComment().isPresent()) {
throw new TrinoException(NOT_SUPPORTED, "This connector does not support creating tables with table comment");
}
return format(
"CREATE TABLE %s (%s) %s",
quoted(remoteTableName),
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_JOIN_PUSHDOWN_WITH_DISTINCT_FROM:
return false;

case SUPPORTS_CREATE_TABLE_WITH_TABLE_COMMENT:
return false;

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 @@ -83,6 +83,7 @@
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_DATA;
import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_CREATE_TABLE_WITH_TABLE_COMMENT;
import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_CREATE_VIEW;
import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_DELETE;
import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_DROP_COLUMN;
Expand Down Expand Up @@ -1891,6 +1892,24 @@ public void testCreateTable()
assertFalse(getQueryRunner().tableExists(getSession(), tableNameLike));
}

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

String tableName = "test_create_" + randomTableSuffix();

if (!hasBehavior(SUPPORTS_CREATE_TABLE_WITH_TABLE_COMMENT)) {
assertQueryFails("CREATE TABLE " + tableName + " (a bigint) COMMENT 'test comment'", "This connector does not support creating tables with table comment");
return;
}

assertUpdate("CREATE TABLE " + tableName + " (a bigint) COMMENT 'test comment'");
assertEquals(getTableComment(getSession().getCatalog().orElseThrow(), getSession().getSchema().orElseThrow(), tableName), "test comment");

assertUpdate("DROP TABLE " + tableName);
}

@Test
public void testCreateTableSchemaNotFound()
{
Expand Down Expand Up @@ -1980,6 +1999,24 @@ public void testCreateTableAsSelect()
assertUpdate("DROP TABLE " + tableName);
}

@Test
public void testCreateTableAsSelectWithTableComment()
{
skipTestUnless(hasBehavior(SUPPORTS_CREATE_TABLE_WITH_DATA));

String tableName = "test_ctas_" + randomTableSuffix();

if (!hasBehavior(SUPPORTS_CREATE_TABLE_WITH_TABLE_COMMENT)) {
assertQueryFails("CREATE TABLE " + tableName + " COMMENT 'test comment' AS SELECT name FROM nation", "This connector does not support creating tables with table comment");
return;
}

assertUpdate("CREATE TABLE " + tableName + " COMMENT 'test comment' AS SELECT name FROM nation", 25);
assertEquals(getTableComment(getSession().getCatalog().orElseThrow(), getSession().getSchema().orElseThrow(), tableName), "test comment");

assertUpdate("DROP TABLE " + tableName);
}

@Test
public void testCreateTableAsSelectSchemaNotFound()
{
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_TABLE_COMMENT(SUPPORTS_CREATE_TABLE),
SUPPORTS_RENAME_TABLE,
SUPPORTS_RENAME_TABLE_ACROSS_SCHEMAS(SUPPORTS_RENAME_TABLE),

Expand Down

0 comments on commit bb0d814

Please sign in to comment.