Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
14 changes: 14 additions & 0 deletions plugin/trino-prometheus/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -166,6 +166,20 @@
</exclusions>
</dependency>

<dependency>
<groupId>io.trino</groupId>
<artifactId>trino-main</artifactId>
<type>test-jar</type>
<scope>test</scope>

<exclusions>
<exclusion>
<groupId>commons-codec</groupId>
<artifactId>commons-codec</artifactId>
</exclusion>
</exclusions>
</dependency>

<dependency>
<groupId>io.trino</groupId>
<artifactId>trino-testing</artifactId>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,6 @@
import java.net.URI;
import java.time.Duration;

import static io.trino.plugin.prometheus.PrometheusClient.METRICS_ENDPOINT;
import static org.testcontainers.utility.MountableFile.forClasspathResource;

public class PrometheusServer
Expand All @@ -32,6 +31,7 @@ public class PrometheusServer

public static final String USER = "admin";
public static final String PASSWORD = "password";
public static final String PROMETHEUS_QUERY_API = "/api/v1/query?query=up[1d]";

private final GenericContainer<?> dockerContainer;

Expand All @@ -44,14 +44,14 @@ public PrometheusServer(String version, boolean enableBasicAuth)
{
this.dockerContainer = new GenericContainer<>("prom/prometheus:" + version)
.withExposedPorts(PROMETHEUS_PORT)
.waitingFor(Wait.forHttp(METRICS_ENDPOINT).forResponsePredicate(response -> response.contains("\"up\"")))
.withStartupTimeout(Duration.ofSeconds(120));
.waitingFor(Wait.forHttp(PROMETHEUS_QUERY_API).forResponsePredicate(response -> response.contains("\"values\"")))
.withStartupTimeout(Duration.ofSeconds(360));
// Basic authentication was introduced in v2.24.0
if (enableBasicAuth) {
this.dockerContainer
.withCommand("--config.file=/etc/prometheus/prometheus.yml", "--web.config.file=/etc/prometheus/web.yml")
.withCopyFileToContainer(forClasspathResource("web.yml"), "/etc/prometheus/web.yml")
.waitingFor(Wait.forHttp(METRICS_ENDPOINT).forResponsePredicate(response -> response.contains("\"up\"")).withBasicCredentials(USER, PASSWORD))
.waitingFor(Wait.forHttp(PROMETHEUS_QUERY_API).forResponsePredicate(response -> response.contains("\"values\"")).withBasicCredentials(USER, PASSWORD))
Comment thread
ebyhr marked this conversation as resolved.
Outdated
.withStartupTimeout(Duration.ofSeconds(360));
}
this.dockerContainer.start();
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,128 @@
/*
* 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.plugin.prometheus;

import com.google.common.collect.ImmutableMap;
import io.airlift.units.Duration;
import io.trino.spi.connector.ConnectorSplitSource;
import io.trino.spi.connector.DynamicFilter;
import io.trino.testing.AbstractTestQueryFramework;
import io.trino.testing.MaterializedResult;
import io.trino.testing.QueryRunner;
import org.testng.annotations.Test;

import java.util.concurrent.TimeUnit;

import static io.trino.plugin.prometheus.PrometheusQueryRunner.createPrometheusClient;
import static io.trino.plugin.prometheus.PrometheusQueryRunner.createPrometheusQueryRunner;
import static io.trino.spi.connector.NotPartitionedPartitionHandle.NOT_PARTITIONED;
import static java.util.concurrent.TimeUnit.DAYS;
import static java.util.concurrent.TimeUnit.SECONDS;
import static org.assertj.core.api.Assertions.assertThat;
import static org.testng.Assert.assertEquals;

public class TestPrometheusIntegration
extends AbstractTestQueryFramework
{
private static final int NUMBER_MORE_THAN_EXPECTED_NUMBER_SPLITS = 100;

private PrometheusServer server;
private PrometheusClient client;

@Override
protected QueryRunner createQueryRunner()
throws Exception
{
this.server = closeAfterClass(new PrometheusServer());
this.client = createPrometheusClient(server);
return createPrometheusQueryRunner(server, ImmutableMap.of(), ImmutableMap.of());
}

@Test
public void testSelectTable()
{
assertThat(query("SELECT labels FROM prometheus.default.up LIMIT 1"))
.matches("SELECT MAP(ARRAY[VARCHAR 'instance', '__name__', 'job'], ARRAY[VARCHAR 'localhost:9090', 'up', 'prometheus'])");
}

@Test
public void testPushDown()
{
// default interval on the `up` metric that Prometheus records on itself is about 15 seconds, so this should only yield one or two row
MaterializedResult results = computeActual("SELECT * FROM prometheus.default.up WHERE timestamp > (NOW() - INTERVAL '15' SECOND)");
assertThat(results).hasSizeBetween(1, 2);
}

@Test
public void testShowTables()
{
assertQuery("SHOW TABLES IN default LIKE 'up'", "VALUES 'up'");
}

@Test
public void testShowCreateSchema()
{
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()
{
assertQuery("SHOW SCHEMAS LIKE 'default'", "VALUES 'default'");
}

@Test
public void testCreateTable()
{
assertQueryFails("CREATE TABLE default.foo (text VARCHAR)", "This connector does not support creating tables");
}

@Test
public void testDropTable()
{
assertQueryFails("DROP TABLE default.up", "This connector does not support dropping tables");
}

@Test
public void testDescribeTable()
{
assertQuery("DESCRIBE default.up",
"VALUES " +
"('labels', 'map(varchar, varchar)', '', '')," +
"('timestamp', 'timestamp(3) with time zone', '', '')," +
"('value', 'double', '', '')");
}

@Test
public void testCorrectNumberOfSplitsCreated()
{
PrometheusConnectorConfig config = new PrometheusConnectorConfig();
config.setPrometheusURI(server.getUri());
config.setMaxQueryRangeDuration(new Duration(21, DAYS));
config.setQueryChunkSizeDuration(new Duration(1, DAYS));
config.setCacheDuration(new Duration(30, SECONDS));
PrometheusTable table = client.getTable("default", "up");
PrometheusSplitManager splitManager = new PrometheusSplitManager(client, new PrometheusClock(), config);
ConnectorSplitSource splits = splitManager.getSplits(
null,
null,
new PrometheusTableHandle("default", table.getName()),
null,
(DynamicFilter) null);
int numSplits = splits.getNextBatch(NOT_PARTITIONED, NUMBER_MORE_THAN_EXPECTED_NUMBER_SPLITS).getNow(null).getSplits().size();
assertEquals(numSplits, config.getMaxQueryRangeDuration().getValue(TimeUnit.SECONDS) / config.getQueryChunkSizeDuration().getValue(TimeUnit.SECONDS),
0.001);
}
}

This file was deleted.

Loading