Skip to content

Commit

Permalink
Move explanation of TrinoStatus to its own section in routing-rules page
Browse files Browse the repository at this point in the history
  • Loading branch information
andythsu committed Oct 8, 2024
1 parent 1f61b2d commit 2b76c56
Show file tree
Hide file tree
Showing 3 changed files with 20 additions and 11 deletions.
15 changes: 15 additions & 0 deletions docs/routing-rules.md
Original file line number Diff line number Diff line change
Expand Up @@ -134,6 +134,21 @@ condition: 'request.getHeader("X-Trino-Client-Tags") contains "label=foo"'

If no rules match, then request is routed to adhoc.

### TrinoStatus

This class attempts to track the current state of Trino cluster. It is updated per every healthcheck.
There are three possible states

- PENDING
- A Trino cluster will show this state when it is still starting up. It will be treated as
unhealthy by RoutingManager, and therefore requests will not be routed to PENDING clusters
- HEALTHY
- A Trino cluster will show this state when healthchecks report clusters as healthy and ready.
RoutingManager will only route requests to healthy clusters
- UNHEALTHY
- A Trino cluster will show this state when healthchecks report clusters as unhealthy. RoutingManager
will not route requests to unhealthy clusters.

### TrinoRequestUser

This class attempts to extract the user from a request. In order, it attempts
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -13,12 +13,6 @@
*/
package io.trino.gateway.ha.clustermonitor;

/**
* PENDING is for ui/observability purpose and functionally it's unhealthy
* We should use PENDING when Trino clusters are still spinning up
* HEALTHY is when health checks report clusters as up
* UNHEALTHY is when health checks report clusters as down
*/
public enum TrinoStatus
{
PENDING,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -146,12 +146,12 @@ public static void setUpBackend(
.build();
Response response = httpClient.newCall(request).execute();
assertThat(response.isSuccessful()).isTrue();
TrinoStatus newClusterHealthState = TrinoStatus.PENDING;
TrinoStatus newTrinoStatus = TrinoStatus.PENDING;
Stopwatch stopwatch = Stopwatch.createStarted();
// pull cluster health states for 10 seconds
// It should be enough as the healthcheck is run every second
Duration timeout = Duration.ofSeconds(10);
while (newClusterHealthState != TrinoStatus.HEALTHY && stopwatch.elapsed().compareTo(timeout) < 0) {
while (newTrinoStatus != TrinoStatus.HEALTHY && stopwatch.elapsed().compareTo(timeout) < 0) {
// check the state of newly added cluster every second
Request getBackendStateRequest = new Request.Builder()
.url(format("http://localhost:%s/api/public/backends/%s/state", routerPort, name))
Expand All @@ -163,14 +163,14 @@ public static void setUpBackend(
ResponseBody getBackendStateResponseBody = getBackendStateResponse.body();
if (getBackendStateResponseBody != null) {
ClusterStats clusterStats = responseCodec.fromJson(getBackendStateResponseBody.string());
newClusterHealthState = clusterStats.healthState();
log.debug("health state for cluster %s is %s", name, newClusterHealthState);
newTrinoStatus = clusterStats.healthStatus();
log.debug("health state for cluster %s is %s", name, newTrinoStatus);
}
}
}
Thread.sleep(1000);
}
assertThat(newClusterHealthState).isEqualTo(TrinoStatus.HEALTHY);
assertThat(newTrinoStatus).isEqualTo(TrinoStatus.HEALTHY);
}

public record TestConfig(String configFilePath, String h2DbFilePath)
Expand Down

0 comments on commit 2b76c56

Please sign in to comment.