Skip to content

Commit

Permalink
Fix error for filtered table listings
Browse files Browse the repository at this point in the history
  • Loading branch information
electrum authored and findepi committed Mar 15, 2022
1 parent 0d1e5cc commit d5e532b
Show file tree
Hide file tree
Showing 3 changed files with 22 additions and 24 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -499,10 +499,10 @@ public List<QualifiedObjectName> listTables(Session session, QualifiedTablePrefi

Optional<QualifiedObjectName> objectName = prefix.asQualifiedObjectName();
if (objectName.isPresent()) {
if (isExistingRelation(session, objectName.get())) {
return ImmutableList.of(objectName.get());
Optional<Boolean> exists = isExistingRelationForListing(session, objectName.get());
if (exists.isPresent()) {
return exists.get() ? ImmutableList.of(objectName.get()) : ImmutableList.of();
}
return ImmutableList.of();
}

Optional<CatalogMetadata> catalog = getOptionalCatalogMetadata(session, prefix.getCatalogName());
Expand All @@ -522,20 +522,27 @@ public List<QualifiedObjectName> listTables(Session session, QualifiedTablePrefi
return ImmutableList.copyOf(tables);
}

private boolean isExistingRelation(Session session, QualifiedObjectName name)
private Optional<Boolean> isExistingRelationForListing(Session session, QualifiedObjectName name)
{
if (isMaterializedView(session, name)) {
return true;
return Optional.of(true);
}
if (isView(session, name)) {
return true;
return Optional.of(true);
}

// If the table is not redirected, table handle existence is checked.
// If the table is redirected, the target table handle is retrieved. If it does not exist, an
// exception is thrown. This behavior is currently inconsistent with the unfiltered case of table listing.
// TODO: the behavior may change with a different way to resolve relation names. https://github.com/trinodb/trino/issues/9400
return getRedirectionAwareTableHandle(session, name).getTableHandle().isPresent();
// TODO: consider a better way to resolve relation names: https://github.com/trinodb/trino/issues/9400
try {
return Optional.of(getRedirectionAwareTableHandle(session, name).getTableHandle().isPresent());
}
catch (TrinoException e) {
// ignore redirection errors for consistency with listing
if (e.getErrorCode().equals(TABLE_REDIRECTION_ERROR.toErrorCode())) {
return Optional.of(true);
}
// we don't know if it exists or not
return Optional.empty();
}
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -40,16 +40,13 @@
import java.util.Map;
import java.util.stream.IntStream;

import static io.airlift.units.Duration.succinctDuration;
import static io.confluent.kafka.serializers.AbstractKafkaAvroSerDeConfig.SCHEMA_REGISTRY_URL_CONFIG;
import static io.confluent.kafka.serializers.AbstractKafkaAvroSerDeConfig.VALUE_SUBJECT_NAME_STRATEGY;
import static io.trino.testing.assertions.Assert.assertEventually;
import static io.trino.testing.sql.TestTable.randomTableSuffix;
import static java.lang.Math.multiplyExact;
import static java.lang.String.format;
import static java.util.Locale.ENGLISH;
import static java.util.Objects.requireNonNull;
import static java.util.concurrent.TimeUnit.SECONDS;
import static org.apache.kafka.clients.producer.ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG;
import static org.apache.kafka.clients.producer.ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG;
import static org.assertj.core.api.Assertions.assertThat;
Expand Down Expand Up @@ -186,13 +183,9 @@ public void testUnsupportedFormat()
.put(VALUE_SERIALIZER_CLASS_CONFIG, KafkaJsonSchemaSerializer.class.getName())
.buildOrThrow());

String errorMessage = "Not supported schema: JSON";
assertEventually(
succinctDuration(10, SECONDS),
() -> assertThatThrownBy(() -> tableExists(topicName))
.isInstanceOf(RuntimeException.class)
.hasMessage(errorMessage));
assertTrue(tableExists(topicName));

String errorMessage = "Not supported schema: JSON";
assertThatThrownBy(() -> getQueryRunner().execute("SHOW COLUMNS FROM " + toDoubleQuoted(topicName)))
.hasMessage(errorMessage);
assertThatThrownBy(() -> getQueryRunner().execute("SELECT * FROM " + toDoubleQuoted(topicName)))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -282,16 +282,14 @@ public void testTableListing()
VALID_REDIRECTION_SRC),
format("VALUES ('%s', '%s')", SCHEMA_ONE, VALID_REDIRECTION_SRC));

assertQueryFails(
assertQuery(
format("SELECT table_schema, table_name"
+ " FROM information_schema.tables"
+ " WHERE table_catalog='%s' AND table_schema = '%s' AND table_name='%s'",
CATALOG_NAME,
SCHEMA_ONE,
BAD_REDIRECTION_SRC),
format("Table '%1$s' redirected to '%2$s', but the target table '%2$s' does not exist",
new CatalogSchemaTableName(CATALOG_NAME, SCHEMA_ONE, BAD_REDIRECTION_SRC),
new CatalogSchemaTableName(CATALOG_NAME, SCHEMA_TWO, NON_EXISTENT_TABLE)));
format("VALUES ('%s', '%s')", SCHEMA_ONE, BAD_REDIRECTION_SRC));

assertQuery(format(
"SELECT table_schema, table_name"
Expand Down

0 comments on commit d5e532b

Please sign in to comment.