Skip to content

Commit

Permalink
Fix JSON serialization of QualifiedObjectName
Browse files Browse the repository at this point in the history
The QualifiedObjectName with components containing dots did not
round-trip.

As a desired side effect, format table names in error messages less
ambiguously: previously, when table `"a.b.c"."d.e"."f.e.g"` was not
found, the message would indicate a `a.b.c.d.e.f.e.g` table without a
hint how to parse that.
  • Loading branch information
findepi committed Nov 7, 2023
1 parent 47d7d3e commit 794ca2a
Show file tree
Hide file tree
Showing 31 changed files with 165 additions and 72 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -15,8 +15,6 @@

import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonValue;
import com.google.common.base.Splitter;
import com.google.common.collect.ImmutableList;
import com.google.errorprone.annotations.Immutable;
import io.trino.spi.connector.CatalogSchemaRoutineName;
import io.trino.spi.connector.CatalogSchemaTableName;
Expand All @@ -26,6 +24,8 @@

import java.util.Objects;
import java.util.function.Function;
import java.util.regex.Matcher;
import java.util.regex.Pattern;

import static com.google.common.base.Preconditions.checkArgument;
import static io.trino.metadata.MetadataUtil.checkObjectName;
Expand All @@ -34,15 +34,17 @@
@Immutable
public class QualifiedObjectName
{
private static final Pattern UNQUOTED_COMPONENT = Pattern.compile("[a-zA-Z0-9_]+");
private static final String COMPONENT = UNQUOTED_COMPONENT.pattern() + "|\"([^\"]|\"\")*\"";
private static final Pattern PATTERN = Pattern.compile("(?<catalog>" + COMPONENT + ")\\.(?<schema>" + COMPONENT + ")\\.(?<table>" + COMPONENT + ")");

@JsonCreator
public static QualifiedObjectName valueOf(String name)
{
requireNonNull(name, "name is null");

ImmutableList<String> ids = ImmutableList.copyOf(Splitter.on('.').split(name));
checkArgument(ids.size() == 3, "Invalid name %s", name);

return new QualifiedObjectName(ids.get(0), ids.get(1), ids.get(2));
Matcher matcher = PATTERN.matcher(name);
checkArgument(matcher.matches(), "Invalid name %s", name);
return new QualifiedObjectName(unquoteIfNeeded(matcher.group("catalog")), unquoteIfNeeded(matcher.group("schema")), unquoteIfNeeded(matcher.group("table")));
}

private final String catalogName;
Expand Down Expand Up @@ -127,11 +129,28 @@ public int hashCode()
@Override
public String toString()
{
return catalogName + '.' + schemaName + '.' + objectName;
return quoteIfNeeded(catalogName) + '.' + quoteIfNeeded(schemaName) + '.' + quoteIfNeeded(objectName);
}

public static Function<SchemaTableName, QualifiedObjectName> convertFromSchemaTableName(String catalogName)
{
return input -> new QualifiedObjectName(catalogName, input.getSchemaName(), input.getTableName());
}

private static String unquoteIfNeeded(String name)
{
if (name.isEmpty() || name.charAt(0) != '"') {
return name;
}
checkArgument(name.charAt(name.length() - 1) == '"', "Invalid name: [%s]", name);
return name.substring(1, name.length() - 1).replace("\"\"", "\"");
}

private static String quoteIfNeeded(String name)
{
if (UNQUOTED_COMPONENT.matcher(name).matches()) {
return name;
}
return "\"" + name.replace("\"", "\"\"") + "\"";
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,7 @@ public final class TestingHandles
private TestingHandles() {}

private static final CatalogVersion TEST_CATALOG_VERSION = new CatalogVersion("test");
public static final String TEST_CATALOG_NAME = "test-catalog";
public static final String TEST_CATALOG_NAME = "test_catalog";
public static final CatalogHandle TEST_CATALOG_HANDLE = createTestCatalogHandle(TEST_CATALOG_NAME);
public static final TableHandle TEST_TABLE_HANDLE = new TableHandle(
TEST_CATALOG_HANDLE,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -104,7 +104,7 @@ public void testExecuteNoPermission()
assertThatThrownBy(
() -> executeCallTask(PROCEDURE_METHOD_HANDLE.bindTo(target), transactionManager -> new DenyAllAccessControl()))
.isInstanceOf(AccessDeniedException.class)
.hasMessage("Access Denied: Cannot execute procedure test-catalog.test.testing_procedure");
.hasMessage("Access Denied: Cannot execute procedure test_catalog.test.testing_procedure");
assertThat(target.invoked).isFalse();
}

Expand All @@ -121,7 +121,7 @@ public void testExecuteNoPermissionOnInsert()
return accessControl;
}))
.isInstanceOf(AccessDeniedException.class)
.hasMessage("Access Denied: Cannot insert into table test-catalog.test.testing_table");
.hasMessage("Access Denied: Cannot insert into table test_catalog.test.testing_table");
}

private void executeCallTask(MethodHandle methodHandle, Function<TransactionManager, AccessControl> accessControlProvider)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -227,7 +227,7 @@ public void testCreateMaterializedViewWithInvalidProperty()
assertTrinoExceptionThrownBy(() -> getFutureValue(new CreateMaterializedViewTask(plannerContext, new AllowAllAccessControl(), parser, analyzerFactory, materializedViewPropertyManager)
.execute(statement, queryStateMachine, ImmutableList.of(), WarningCollector.NOOP)))
.hasErrorCode(INVALID_MATERIALIZED_VIEW_PROPERTY)
.hasMessage("Catalog 'test-catalog' materialized view property 'baz' does not exist");
.hasMessage("Catalog 'test_catalog' materialized view property 'baz' does not exist");

assertEquals(metadata.getCreateMaterializedViewCallCount(), 0);
}
Expand Down Expand Up @@ -283,7 +283,7 @@ public void testCreateDenyPermission()
assertThatThrownBy(() -> getFutureValue(new CreateMaterializedViewTask(plannerContext, accessControl, parser, analyzerFactory, materializedViewPropertyManager)
.execute(statement, queryStateMachine, ImmutableList.of(), WarningCollector.NOOP)))
.isInstanceOf(AccessDeniedException.class)
.hasMessageContaining("Cannot create materialized view test-catalog.schema.test_mv");
.hasMessageContaining("Cannot create materialized view test_catalog.schema.test_mv");
}

private QueryStateMachine stateMachine(TransactionManager transactionManager, MetadataManager metadata, AccessControl accessControl)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -46,7 +46,7 @@ public void testDuplicatedCreateSchema()
assertTrue(metadata.schemaExists(testSession, CATALOG_SCHEMA_NAME));
assertThatExceptionOfType(TrinoException.class)
.isThrownBy(() -> getFutureValue(task.execute(statement, queryStateMachine, emptyList(), WarningCollector.NOOP)))
.withMessage("Schema 'test-catalog.test_db' already exists");
.withMessage("Schema 'test_catalog.test_db' already exists");
}

@Test
Expand All @@ -71,14 +71,14 @@ public void failCreateSchema()
queryStateMachine,
emptyList(),
WarningCollector.NOOP)))
.withMessage("TEST create schema fail: test-catalog.test_db");
.withMessage("TEST create schema fail: test_catalog.test_db");
assertThatExceptionOfType(TrinoException.class)
.isThrownBy(() -> getFutureValue(task.execute(
new CreateSchema(QualifiedName.of(CATALOG_SCHEMA_NAME.getSchemaName()), true, ImmutableList.of()),
queryStateMachine,
emptyList(),
WarningCollector.NOOP)))
.withMessage("TEST create schema fail: test-catalog.test_db");
.withMessage("TEST create schema fail: test_catalog.test_db");
}

private CreateSchemaTask getCreateSchemaTask()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -224,7 +224,7 @@ public void testCreateTableWithMaterializedViewPropertyFails()
CreateTableTask createTableTask = new CreateTableTask(plannerContext, new AllowAllAccessControl(), columnPropertyManager, tablePropertyManager);
assertTrinoExceptionThrownBy(() -> getFutureValue(createTableTask.internalExecute(statement, testSession, emptyList(), output -> {})))
.hasErrorCode(INVALID_TABLE_PROPERTY)
.hasMessage("Catalog 'test-catalog' table property 'foo' does not exist");
.hasMessage("Catalog 'test_catalog' table property 'foo' does not exist");

assertEquals(metadata.getCreateTableCallCount(), 0);
}
Expand Down Expand Up @@ -276,7 +276,7 @@ public void testCreateWithUnsupportedConnectorThrowsWhenNotNull()
assertTrinoExceptionThrownBy(() ->
getFutureValue(createTableTask.internalExecute(statement, testSession, emptyList(), output -> {})))
.hasErrorCode(NOT_SUPPORTED)
.hasMessage("Catalog 'test-catalog' does not support non-null column for column name 'b'");
.hasMessage("Catalog 'test_catalog' does not support non-null column for column name 'b'");
}

@Test
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -56,7 +56,7 @@ public void testDropSchemaRestrict()

assertThatExceptionOfType(TrinoException.class)
.isThrownBy(() -> getFutureValue(dropSchemaTask.execute(dropSchema, queryStateMachine, emptyList(), NOOP)))
.withMessage("Schema 'test-catalog.test_db' does not exist");
.withMessage("Schema 'test_catalog.test_db' does not exist");
}

@Test
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -107,7 +107,7 @@ public void testRenameMaterializedViewOnView()

assertTrinoExceptionThrownBy(() -> getFutureValue(executeRenameMaterializedView(viewName, qualifiedName("existing_view_new"))))
.hasErrorCode(TABLE_NOT_FOUND)
.hasMessage("Materialized View '%s' does not exist, but a view with that name exists. Did you mean ALTER VIEW test-catalog.schema.existing_view RENAME TO ...?", viewName);
.hasMessage("Materialized View '%s' does not exist, but a view with that name exists. Did you mean ALTER VIEW test_catalog.schema.existing_view RENAME TO ...?", viewName);
}

@Test
Expand All @@ -118,7 +118,7 @@ public void testRenameMaterializedViewOnViewIfExists()

assertTrinoExceptionThrownBy(() -> getFutureValue(executeRenameMaterializedView(viewName, qualifiedName("existing_view_new"), true)))
.hasErrorCode(TABLE_NOT_FOUND)
.hasMessage("Materialized View '%s' does not exist, but a view with that name exists. Did you mean ALTER VIEW test-catalog.schema.existing_view RENAME TO ...?", viewName);
.hasMessage("Materialized View '%s' does not exist, but a view with that name exists. Did you mean ALTER VIEW test_catalog.schema.existing_view RENAME TO ...?", viewName);
}

@Test
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -94,7 +94,7 @@ public void testRenameTableOnMaterializedView()

assertTrinoExceptionThrownBy(() -> getFutureValue(executeRenameTable(viewName, qualifiedName("existing_materialized_view_new"), false)))
.hasErrorCode(GENERIC_USER_ERROR)
.hasMessage("Table '%s' does not exist, but a materialized view with that name exists. Did you mean ALTER MATERIALIZED VIEW test-catalog.schema.existing_materialized_view RENAME TO ...?", viewName);
.hasMessage("Table '%s' does not exist, but a materialized view with that name exists. Did you mean ALTER MATERIALIZED VIEW test_catalog.schema.existing_materialized_view RENAME TO ...?", viewName);
}

@Test
Expand All @@ -105,7 +105,7 @@ public void testRenameTableOnMaterializedViewIfExists()

assertTrinoExceptionThrownBy(() -> getFutureValue(executeRenameTable(viewName, qualifiedName("existing_materialized_view_new"), true)))
.hasErrorCode(GENERIC_USER_ERROR)
.hasMessage("Table '%s' does not exist, but a materialized view with that name exists. Did you mean ALTER MATERIALIZED VIEW test-catalog.schema.existing_materialized_view RENAME TO ...?", viewName);
.hasMessage("Table '%s' does not exist, but a materialized view with that name exists. Did you mean ALTER MATERIALIZED VIEW test_catalog.schema.existing_materialized_view RENAME TO ...?", viewName);
}

@Test
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -75,7 +75,7 @@ public void testRenameViewOnMaterializedView()

assertTrinoExceptionThrownBy(() -> getFutureValue(executeRenameView(viewName, qualifiedName("existing_materialized_view_new"))))
.hasErrorCode(TABLE_NOT_FOUND)
.hasMessage("View '%s' does not exist, but a materialized view with that name exists. Did you mean ALTER MATERIALIZED VIEW test-catalog.schema.existing_materialized_view RENAME TO ...?", viewName);
.hasMessage("View '%s' does not exist, but a materialized view with that name exists. Did you mean ALTER MATERIALIZED VIEW test_catalog.schema.existing_materialized_view RENAME TO ...?", viewName);
}

@Test
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,53 @@
/*
* 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.metadata;

import io.airlift.json.JsonCodec;
import org.junit.jupiter.api.Test;

import static org.assertj.core.api.Assertions.assertThatThrownBy;
import static org.testng.Assert.assertEquals;

public class TestQualifiedObjectName
{
private final JsonCodec<QualifiedObjectName> codec = JsonCodec.jsonCodec(QualifiedObjectName.class);

@Test
public void testJsonSerializationRoundTrip()
{
// simple
testRoundTrip(new QualifiedObjectName("catalog", "schema", "table_name"));

// names with dots
testRoundTrip(new QualifiedObjectName("catalog.twój", "schema.ściema", "tabel.tabelkówna"));

// names with apostrophes
testRoundTrip(new QualifiedObjectName("cata\"l.o.g\"", "s\"ch.e.ma\"", "\"t.a.b.e.l\""));

// non-lowercase (currently illegal but TODO coming in https://github.com/trinodb/trino/issues/17)
assertThatThrownBy(() -> new QualifiedObjectName("CataLOG", "SchemA", "TabEl"))
.isInstanceOf(IllegalArgumentException.class)
.hasMessage("catalogName is not lowercase: CataLOG");

// empty
testRoundTrip(new QualifiedObjectName("", "", ""));
}

private void testRoundTrip(QualifiedObjectName value)
{
String json = codec.toJson(value);
QualifiedObjectName parsed = codec.fromJson(json);
assertEquals(parsed, value);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -136,7 +136,7 @@ public void testReadOnlySystemAccessControl()
accessControlManager.checkCanInsertIntoTable(new SecurityContext(transactionId, identity, queryId, queryStart), tableName);
}))
.isInstanceOf(AccessDeniedException.class)
.hasMessage("Access Denied: Cannot insert into table test-catalog.schema.table");
.hasMessage("Access Denied: Cannot insert into table test_catalog.schema.table");
}

@Test
Expand Down Expand Up @@ -311,7 +311,7 @@ public void testDenyExecuteProcedureBySystem()
accessControlManager.addSystemAccessControlFactory(accessControlFactory);
accessControlManager.loadSystemAccessControl("deny-all", ImmutableMap.of());

assertDenyExecuteProcedure(transactionManager, metadata, accessControlManager, "Access Denied: Cannot execute procedure test-catalog.schema.procedure");
assertDenyExecuteProcedure(transactionManager, metadata, accessControlManager, "Access Denied: Cannot execute procedure test_catalog.schema.procedure");
}

@Test
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -944,11 +944,11 @@ public void testInvalidTable()
// table name containing dots
assertFails("SELECT * FROM \"table.not.existing\"")
.hasErrorCode(TABLE_NOT_FOUND)
.hasMessage("line 1:15: Table 'tpch.s1.table.not.existing' does not exist");
.hasMessage("line 1:15: Table 'tpch.s1.\"table.not.existing\"' does not exist");
// table name containing whitespace
assertFails("SELECT * FROM \"table' does not exist, or maybe 'view\"")
.hasErrorCode(TABLE_NOT_FOUND)
.hasMessage("line 1:15: Table 'tpch.s1.table' does not exist, or maybe 'view' does not exist");
.hasMessage("line 1:15: Table 'tpch.s1.\"table' does not exist, or maybe 'view\"' does not exist");
}

@Test
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -50,4 +50,25 @@ public void testRoundTrip()

assertEquals(actual, expected);
}

@Test
public void testRoundWithComplexIdentifiers()
{
Output expected = new Output(
"catalog.Mój",
new CatalogVersion("default"),
"ści.e-Ma",
"ta.b-Elką go",
Optional.of(
ImmutableList.of(
new OutputColumn(
new Column("ko.LU-mieńka", "type"),
ImmutableSet.of(
new SourceColumn(new QualifiedObjectName("catalog.twój", "schema.ściema", "tabel.tabelkówna"), "co-lumn.słodziak\""))))));

String json = codec.toJson(expected);
Output actual = codec.fromJson(json);

assertEquals(actual, expected);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -2028,7 +2028,7 @@ public void testGroupingSetsWithDefaultValue()
public void testSizeBasedJoin()
{
// both local.sf100000.nation and local.sf100000.orders don't provide stats, therefore no reordering happens
assertDistributedPlan("SELECT custkey FROM \"test-catalog\".\"sf42.5\".nation, \"test-catalog\".\"sf42.5\".orders WHERE nation.nationkey = orders.custkey",
assertDistributedPlan("SELECT custkey FROM \"test_catalog\".\"sf42.5\".nation, \"test_catalog\".\"sf42.5\".orders WHERE nation.nationkey = orders.custkey",
automaticJoinDistribution(),
output(
join(INNER, builder -> builder
Expand All @@ -2040,7 +2040,7 @@ public void testSizeBasedJoin()
anyTree(
tableScan("orders", ImmutableMap.of("CUSTKEY", "custkey")))))));

assertDistributedPlan("SELECT custkey FROM (VALUES CAST(1 AS BIGINT), CAST(2 AS BIGINT)) t(a), \"test-catalog\".\"sf42.5\".orders WHERE t.a = orders.custkey",
assertDistributedPlan("SELECT custkey FROM (VALUES CAST(1 AS BIGINT), CAST(2 AS BIGINT)) t(a), \"test_catalog\".\"sf42.5\".orders WHERE t.a = orders.custkey",
automaticJoinDistribution(),
output(
join(INNER, builder -> builder
Expand All @@ -2058,7 +2058,7 @@ public void testSizeBasedJoin()
public void testSizeBasedSemiJoin()
{
// both local.sf100000.nation and local.sf100000.orders don't provide stats, therefore no reordering happens
assertDistributedPlan("SELECT custkey FROM \"test-catalog\".\"sf42.5\".orders WHERE orders.custkey NOT IN (SELECT nationkey FROM \"test-catalog\".\"sf42.5\".nation)",
assertDistributedPlan("SELECT custkey FROM \"test_catalog\".\"sf42.5\".orders WHERE orders.custkey NOT IN (SELECT nationkey FROM \"test_catalog\".\"sf42.5\".nation)",
automaticJoinDistribution(),
output(
anyTree(
Expand All @@ -2069,7 +2069,7 @@ public void testSizeBasedSemiJoin()
tableScan("nation", ImmutableMap.of("NATIONKEY", "nationkey")))))));

// values node provides stats
assertDistributedPlan("SELECT custkey FROM \"test-catalog\".\"sf42.5\".orders WHERE orders.custkey NOT IN (SELECT t.a FROM (VALUES CAST(1 AS BIGINT), CAST(2 AS BIGINT)) t(a))",
assertDistributedPlan("SELECT custkey FROM \"test_catalog\".\"sf42.5\".orders WHERE orders.custkey NOT IN (SELECT t.a FROM (VALUES CAST(1 AS BIGINT), CAST(2 AS BIGINT)) t(a))",
automaticJoinDistribution(),
output(
anyTree(
Expand Down
Loading

0 comments on commit 794ca2a

Please sign in to comment.