Skip to content

Commit

Permalink
Use AbstractTestQueryFramework in Prometheus
Browse files Browse the repository at this point in the history
* Remove singleThreaded
* Use closeAfterClass
* Remove useless testGetColumnMetadata test
  • Loading branch information
ebyhr committed May 17, 2022
1 parent 264a66f commit 390d4aa
Showing 1 changed file with 26 additions and 103 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -13,139 +13,62 @@
*/
package io.trino.plugin.prometheus;

import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableSet;
import io.trino.spi.TrinoException;
import io.trino.spi.connector.ColumnMetadata;
import io.trino.spi.connector.ConnectorTableMetadata;
import io.trino.spi.connector.RecordSet;
import io.trino.spi.connector.SchemaTableName;
import io.trino.spi.type.DoubleType;
import org.testng.annotations.AfterClass;
import org.testng.annotations.BeforeClass;
import com.google.common.collect.ImmutableMap;
import io.trino.testing.AbstractTestQueryFramework;
import io.trino.testing.QueryRunner;
import org.testng.annotations.Test;

import static io.trino.plugin.prometheus.PrometheusClient.TIMESTAMP_COLUMN_TYPE;
import static io.trino.plugin.prometheus.PrometheusQueryRunner.createPrometheusClient;
import static io.trino.spi.type.BigintType.BIGINT;
import static io.trino.spi.type.VarcharType.createUnboundedVarcharType;
import static io.trino.testing.TestingConnectorSession.SESSION;
import static org.assertj.core.api.Assertions.assertThatThrownBy;
import static org.testng.Assert.assertEquals;
import static org.testng.Assert.assertNull;
import static org.testng.Assert.assertTrue;
import static io.trino.plugin.prometheus.PrometheusQueryRunner.createPrometheusQueryRunner;

/**
* Integration tests against Prometheus container
*/
@Test(singleThreaded = true)
public class TestPrometheusIntegrationMetrics
extends AbstractTestQueryFramework
{
private static final PrometheusTableHandle RUNTIME_DETERMINED_TABLE_HANDLE = new PrometheusTableHandle("default", "up");

private PrometheusServer server;
private PrometheusClient client;

@BeforeClass
public void createQueryRunner()
@Override
protected QueryRunner createQueryRunner()
throws Exception
{
this.server = new PrometheusServer();
this.client = createPrometheusClient(server);
}

@AfterClass(alwaysRun = true)
public final void destroy()
{
server.close();
PrometheusServer server = closeAfterClass(new PrometheusServer());
return createPrometheusQueryRunner(server, ImmutableMap.of(), ImmutableMap.of());
}

@Test
public void testRetrieveUpValue()
public void testShowTables()
{
assertTrue(client.getTableNames("default").contains("up"), "Prometheus' own `up` metric should be available in default");
assertQuery("SHOW TABLES IN default LIKE 'up'", "VALUES 'up'");
}

@Test
public void testHandleErrorResponse()
public void testShowCreateSchema()
{
assertThatThrownBy(() -> client.getTableNames("unknown"))
.isInstanceOf(TrinoException.class)
.hasMessageContaining("Prometheus did no return metrics list (table names)");
PrometheusTable table = client.getTable("unknown", "up");
assertNull(table);
assertQuery("SHOW CREATE SCHEMA default", "VALUES 'CREATE SCHEMA prometheus.default'");
assertQueryFails("SHOW CREATE SCHEMA unknown", ".*Schema 'prometheus.unknown' does not exist");
}

@Test
public void testListSchemaNames()
{
PrometheusMetadata metadata = new PrometheusMetadata(client);
assertEquals(metadata.listSchemaNames(SESSION), ImmutableSet.of("default"));
assertQuery("SHOW SCHEMAS LIKE 'default'", "VALUES 'default'");
}

@Test
public void testGetColumnMetadata()
{
PrometheusMetadata metadata = new PrometheusMetadata(client);
assertEquals(metadata.getColumnMetadata(SESSION, RUNTIME_DETERMINED_TABLE_HANDLE, new PrometheusColumnHandle("text", createUnboundedVarcharType(), 0)),
new ColumnMetadata("text", createUnboundedVarcharType()));

// prometheus connector assumes that the table handle and column handle are
// properly formed, so it will return a metadata object for any
// PrometheusTableHandle and PrometheusColumnHandle passed in. This is on because
// it is not possible for the Trino Metadata system to create the handles
// directly.
}

@Test(expectedExceptions = TrinoException.class)
public void testCreateTable()
{
PrometheusMetadata metadata = new PrometheusMetadata(client);
metadata.createTable(
SESSION,
new ConnectorTableMetadata(
new SchemaTableName("default", "foo"),
ImmutableList.of(new ColumnMetadata("text", createUnboundedVarcharType()))),
false);
assertQueryFails("CREATE TABLE default.foo (text VARCHAR)", "This connector does not support creating tables");
}

@Test(expectedExceptions = TrinoException.class)
public void testDropTableTable()
@Test
public void testDropTable()
{
PrometheusMetadata metadata = new PrometheusMetadata(client);
metadata.dropTable(SESSION, RUNTIME_DETERMINED_TABLE_HANDLE);
assertQueryFails("DROP TABLE default.up", "This connector does not support dropping tables");
}

@Test
public void testGetColumnTypes()
public void testDescribeTable()
{
String dataUri = server.getUri().toString();
RecordSet recordSet = new PrometheusRecordSet(
client,
new PrometheusSplit(dataUri),
ImmutableList.of(
new PrometheusColumnHandle("labels", createUnboundedVarcharType(), 0),
new PrometheusColumnHandle("value", DoubleType.DOUBLE, 1),
new PrometheusColumnHandle("timestamp", TIMESTAMP_COLUMN_TYPE, 2)));
assertEquals(recordSet.getColumnTypes(), ImmutableList.of(createUnboundedVarcharType(), DoubleType.DOUBLE, TIMESTAMP_COLUMN_TYPE));

recordSet = new PrometheusRecordSet(
client,
new PrometheusSplit(dataUri),
ImmutableList.of(
new PrometheusColumnHandle("value", BIGINT, 1),
new PrometheusColumnHandle("text", createUnboundedVarcharType(), 0)));
assertEquals(recordSet.getColumnTypes(), ImmutableList.of(BIGINT, createUnboundedVarcharType()));

recordSet = new PrometheusRecordSet(
client,
new PrometheusSplit(dataUri),
ImmutableList.of(
new PrometheusColumnHandle("value", BIGINT, 1),
new PrometheusColumnHandle("value", BIGINT, 1),
new PrometheusColumnHandle("text", createUnboundedVarcharType(), 0)));
assertEquals(recordSet.getColumnTypes(), ImmutableList.of(BIGINT, BIGINT, createUnboundedVarcharType()));

recordSet = new PrometheusRecordSet(client, new PrometheusSplit(dataUri), ImmutableList.of());
assertEquals(recordSet.getColumnTypes(), ImmutableList.of());
assertQuery("DESCRIBE default.up",
"VALUES " +
"('labels', 'map(varchar, varchar)', '', '')," +
"('timestamp', 'timestamp(3) with time zone', '', '')," +
"('value', 'double', '', '')");
}
}

0 comments on commit 390d4aa

Please sign in to comment.