|
13 | 13 | */ |
14 | 14 | package io.trino.plugin.lakehouse; |
15 | 15 |
|
| 16 | +import com.google.common.collect.ImmutableList; |
16 | 17 | import io.trino.Session; |
| 18 | +import io.trino.connector.MockConnectorFactory; |
| 19 | +import io.trino.connector.MockConnectorPlugin; |
17 | 20 | import io.trino.plugin.hive.containers.Hive3MinioDataLake; |
| 21 | +import io.trino.spi.metrics.Metrics; |
| 22 | +import io.trino.spi.statistics.TableStatistics; |
18 | 23 | import io.trino.testing.BaseConnectorTest; |
19 | 24 | import io.trino.testing.MaterializedResult; |
| 25 | +import io.trino.testing.QueryFailedException; |
20 | 26 | import io.trino.testing.QueryRunner; |
21 | 27 | import io.trino.testing.TestingConnectorBehavior; |
| 28 | +import io.trino.testing.TestingSession; |
22 | 29 | import io.trino.testing.sql.TestTable; |
23 | 30 | import org.junit.jupiter.api.BeforeAll; |
24 | 31 | import org.junit.jupiter.api.Disabled; |
25 | 32 | import org.junit.jupiter.api.Test; |
26 | 33 | import org.junit.jupiter.api.TestInstance; |
27 | 34 |
|
| 35 | +import java.util.Map; |
28 | 36 | import java.util.Optional; |
29 | 37 | import java.util.OptionalInt; |
30 | 38 |
|
| 39 | +import static io.trino.SystemSessionProperties.ITERATIVE_OPTIMIZER_TIMEOUT; |
31 | 40 | import static io.trino.plugin.tpch.TpchMetadata.TINY_SCHEMA_NAME; |
32 | 41 | import static io.trino.spi.type.VarcharType.VARCHAR; |
33 | 42 | import static io.trino.testing.MaterializedResult.resultBuilder; |
|
36 | 45 | import static io.trino.testing.containers.Minio.MINIO_ACCESS_KEY; |
37 | 46 | import static io.trino.testing.containers.Minio.MINIO_REGION; |
38 | 47 | import static io.trino.testing.containers.Minio.MINIO_SECRET_KEY; |
| 48 | +import static java.lang.String.format; |
39 | 49 | import static org.assertj.core.api.Assertions.assertThat; |
40 | 50 | import static org.assertj.core.api.Assertions.assertThatThrownBy; |
| 51 | +import static org.assertj.core.api.Fail.fail; |
41 | 52 | import static org.junit.jupiter.api.Assumptions.abort; |
42 | 53 | import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; |
43 | 54 |
|
@@ -81,6 +92,29 @@ public void setUp() |
81 | 92 | copyTpchTables(getQueryRunner(), "tpch", TINY_SCHEMA_NAME, REQUIRED_TPCH_TABLES); |
82 | 93 | } |
83 | 94 |
|
| 95 | + @BeforeAll |
| 96 | + public void initMockMetricsCatalog() |
| 97 | + { |
| 98 | + QueryRunner queryRunner = getQueryRunner(); |
| 99 | + String mockConnector = "mock_metrics"; |
| 100 | + queryRunner.installPlugin(new MockConnectorPlugin(MockConnectorFactory.builder() |
| 101 | + .withName(mockConnector) |
| 102 | + .withListSchemaNames(_ -> ImmutableList.of("default")) |
| 103 | + .withGetTableStatistics(_ -> { |
| 104 | + try { |
| 105 | + Thread.sleep(110); |
| 106 | + } |
| 107 | + catch (InterruptedException e) { |
| 108 | + Thread.currentThread().interrupt(); |
| 109 | + throw new RuntimeException(e); |
| 110 | + } |
| 111 | + return TableStatistics.empty(); |
| 112 | + }) |
| 113 | + .build())); |
| 114 | + |
| 115 | + queryRunner.createCatalog("mock_metrics", mockConnector); |
| 116 | + } |
| 117 | + |
84 | 118 | @Override |
85 | 119 | protected boolean hasBehavior(TestingConnectorBehavior connectorBehavior) |
86 | 120 | { |
@@ -365,4 +399,36 @@ public void testShowCreateTable() |
365 | 399 | type = 'ICEBERG' |
366 | 400 | )\\E"""); |
367 | 401 | } |
| 402 | + |
| 403 | + @Test |
| 404 | + public void testCatalogMetadataMetrics() |
| 405 | + { |
| 406 | + QueryRunner.MaterializedResultWithPlan result = getQueryRunner().executeWithPlan( |
| 407 | + getSession(), |
| 408 | + "SELECT count(*) FROM region r, nation n WHERE r.regionkey = n.regionkey"); |
| 409 | + Map<String, Metrics> metrics = getCatalogMetadataMetrics(result.queryId()); |
| 410 | + assertCountMetricExists(metrics, "lakehouse", "iceberg.metastore.all.time.total"); |
| 411 | + assertDistributionMetricExists(metrics, "lakehouse", "iceberg.metastore.all.time.distribution"); |
| 412 | + assertCountMetricExists(metrics, "lakehouse", "iceberg.metastore.getTable.time.total"); |
| 413 | + assertDistributionMetricExists(metrics, "lakehouse", "iceberg.metastore.getTable.time.distribution"); |
| 414 | + } |
| 415 | + |
| 416 | + @Test |
| 417 | + public void testCatalogMetadataMetricsWithOptimizerTimeoutExceeded() |
| 418 | + { |
| 419 | + String query = "SELECT count(*) FROM region r, nation n, mock_metrics.default.mock_table m WHERE r.regionkey = n.regionkey"; |
| 420 | + try { |
| 421 | + Session smallOptimizerTimeout = TestingSession.testSessionBuilder(getSession()) |
| 422 | + .setSystemProperty(ITERATIVE_OPTIMIZER_TIMEOUT, "100ms") |
| 423 | + .build(); |
| 424 | + QueryRunner.MaterializedResultWithPlan result = getQueryRunner().executeWithPlan(smallOptimizerTimeout, query); |
| 425 | + fail(format("Expected query to fail: %s [QueryId: %s]", query, result.queryId())); |
| 426 | + } |
| 427 | + catch (QueryFailedException e) { |
| 428 | + assertThat(e.getMessage()).contains("The optimizer exhausted the time limit"); |
| 429 | + Map<String, Metrics> metrics = getCatalogMetadataMetrics(e.getQueryId()); |
| 430 | + assertCountMetricExists(metrics, "lakehouse", "iceberg.metastore.all.time.total"); |
| 431 | + assertCountMetricExists(metrics, "lakehouse", "iceberg.metastore.getTable.time.total"); |
| 432 | + } |
| 433 | + } |
368 | 434 | } |
0 commit comments