Skip to content

Commit

Permalink
Add PENDING type to healthchecks
Browse files Browse the repository at this point in the history
  • Loading branch information
andythsu authored and Andy Su (Apps) committed May 31, 2024
1 parent 53af36e commit f1a2261
Show file tree
Hide file tree
Showing 15 changed files with 108 additions and 34 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -68,6 +68,9 @@ public abstract class BaseApp
{
private static final Logger logger = Logger.get(BaseApp.class);
private final ImmutableList.Builder<Module> appModules = ImmutableList.builder();
// this injector reference is needed to use reflection in
// TestGatewayHaSingleBackend and TestGatewayMultipleBackend
private Injector injector;

private Module newModule(String clazz, HaGatewayConfiguration configuration, Environment environment)
{
Expand Down Expand Up @@ -129,7 +132,7 @@ private void configureGuice(HaGatewayConfiguration configuration, Environment en
{
appModules.add(new MetricRegistryModule(environment.metrics()));
appModules.addAll(addModules(configuration, environment));
Injector injector = Guice.createInjector(appModules.build());
injector = Guice.createInjector(appModules.build());
injector.injectMembers(this);
registerWithInjector(configuration, environment, injector);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,7 @@ public record ClusterStats(
int runningQueryCount,
int queuedQueryCount,
int numWorkerNodes,
boolean healthy,
TrinoHealthStateType healthState,
String proxyTo,
String externalUrl,
String routingGroup,
Expand All @@ -41,7 +41,7 @@ public static final class Builder
private int runningQueryCount;
private int queuedQueryCount;
private int numWorkerNodes;
private boolean healthy;
private TrinoHealthStateType healthState;
private String proxyTo;
private String externalUrl;
private String routingGroup;
Expand Down Expand Up @@ -70,9 +70,9 @@ public Builder numWorkerNodes(int numWorkerNodes)
return this;
}

public Builder healthy(boolean healthy)
public Builder healthy(TrinoHealthStateType healthState)
{
this.healthy = healthy;
this.healthState = healthState;
return this;
}

Expand Down Expand Up @@ -107,7 +107,7 @@ public ClusterStats build()
runningQueryCount,
queuedQueryCount,
numWorkerNodes,
healthy,
healthState,
proxyTo,
externalUrl,
routingGroup,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -71,7 +71,7 @@ public ClusterStats monitor(ProxyBackendConfiguration backend)
.numWorkerNodes(activeWorkers)
.queuedQueryCount((int) result.get("queuedQueries"))
.runningQueryCount((int) result.get("runningQueries"))
.healthy(activeWorkers > 0)
.healthy(activeWorkers > 0 ? TrinoHealthStateType.HEALTHY : TrinoHealthStateType.UNHEALTHY)
.proxyTo(backend.getProxyTo())
.externalUrl(backend.getExternalUrl())
.routingGroup(backend.getRoutingGroup());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -46,19 +46,19 @@ public ClusterStats monitor(ProxyBackendConfiguration backend)
.routingGroup(backend.getRoutingGroup()).build();
}

private boolean isReadyStatus(String baseUrl)
private TrinoHealthStateType isReadyStatus(String baseUrl)
{
Request request = prepareGet()
.setUri(uriBuilderFrom(URI.create(baseUrl)).appendPath("/v1/info").build())
.build();

try {
ServerInfo serverInfo = client.execute(request, SERVER_INFO_JSON_RESPONSE_HANDLER);
return !serverInfo.isStarting();
return serverInfo.isStarting() ? TrinoHealthStateType.PENDING : TrinoHealthStateType.HEALTHY;
}
catch (Exception e) {
log.error("Exception checking {} for health: {} ", request.getUri(), e.getMessage());
}
return false;
return TrinoHealthStateType.UNHEALTHY;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -82,7 +82,9 @@ public ClusterStats monitor(ProxyBackendConfiguration backend)
partialState.put(rs.getString("state"), rs.getInt("count"));
}
return clusterStats
.healthy(true)
// at this point we can set cluster to healthState because otherwise
// it wouldn't have gotten worker stats
.healthy(TrinoHealthStateType.HEALTHY)
.queuedQueryCount(partialState.getOrDefault("QUEUED", 0))
.runningQueryCount(partialState.getOrDefault("RUNNING", 0))
.build();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,7 @@ public HealthChecker(Notifier notifier)
public void observe(List<ClusterStats> clustersStats)
{
for (ClusterStats clusterStats : clustersStats) {
if (!clusterStats.healthy()) {
if (clusterStats.healthState() == TrinoHealthStateType.UNHEALTHY) {
notifyUnhealthyCluster(clusterStats);
}
else {
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,27 @@
/*
* 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.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 TrinoHealthStateType
{
PENDING,
HEALTHY,
UNHEALTHY
}
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,10 @@
import com.google.common.collect.ImmutableList;
import com.google.inject.Inject;
import io.airlift.log.Logger;
import io.trino.gateway.ha.clustermonitor.ClusterStats;
import io.trino.gateway.ha.clustermonitor.TrinoHealthStateType;
import io.trino.gateway.ha.config.ProxyBackendConfiguration;
import io.trino.gateway.ha.router.BackendStateManager;
import io.trino.gateway.ha.router.GatewayBackendManager;
import io.trino.gateway.ha.router.ResourceGroupsManager;
import io.trino.gateway.ha.router.RoutingManager;
Expand Down Expand Up @@ -52,13 +55,19 @@ public class EntityEditorResource
private final GatewayBackendManager gatewayBackendManager;
private final ResourceGroupsManager resourceGroupsManager;
private final RoutingManager routingManager;
private final BackendStateManager backendStateManager;

@Inject
public EntityEditorResource(GatewayBackendManager gatewayBackendManager, ResourceGroupsManager resourceGroupsManager, RoutingManager routingManager)
public EntityEditorResource(
GatewayBackendManager gatewayBackendManager,
ResourceGroupsManager resourceGroupsManager,
RoutingManager routingManager,
BackendStateManager backendStateManager)
{
this.gatewayBackendManager = requireNonNull(gatewayBackendManager, "gatewayBackendManager is null");
this.resourceGroupsManager = requireNonNull(resourceGroupsManager, "resourceGroupsManager is null");
this.routingManager = requireNonNull(routingManager, "routingManager is null");
this.backendStateManager = requireNonNull(backendStateManager, "backendStateManager is null");
}

@GET
Expand Down Expand Up @@ -88,6 +97,11 @@ public Response updateEntity(
gatewayBackendManager.updateBackend(backend);
log.info("Turning cluster %s %s", backend.getName(), backend.isActive() ? "on" : "off");
routingManager.updateBackEndHealth(backend.getName(), backend.isActive());
backendStateManager.updateStates(
backend.getName(),
ClusterStats.builder(backend.getName())
.healthy(backend.isActive() ? TrinoHealthStateType.PENDING : TrinoHealthStateType.UNHEALTHY)
.build());
break;
case RESOURCE_GROUP:
ResourceGroupsDetail resourceGroupDetails = OBJECT_MAPPER.readValue(jsonPayload,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@
import com.google.common.collect.ImmutableList;
import com.google.errorprone.annotations.concurrent.GuardedBy;
import io.trino.gateway.ha.clustermonitor.ClusterStats;
import io.trino.gateway.ha.clustermonitor.TrinoHealthStateType;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

Expand Down Expand Up @@ -46,7 +47,7 @@ static class LocalStats
{
private int runningQueryCount;
private int queuedQueryCount;
private boolean healthy;
private TrinoHealthStateType healthState;
private String proxyTo;
private String routingGroup;
private String clusterId;
Expand All @@ -57,7 +58,7 @@ static class LocalStats
clusterId = stats.clusterId();
runningQueryCount = stats.runningQueryCount();
queuedQueryCount = stats.queuedQueryCount();
healthy = stats.healthy();
healthState = stats.healthState();
proxyTo = stats.proxyTo();
routingGroup = stats.routingGroup();
if (stats.userQueuedCount() != null) {
Expand Down Expand Up @@ -93,14 +94,14 @@ public void queuedQueryCount(int queuedQueryCount)
this.queuedQueryCount = queuedQueryCount;
}

public boolean healthy()
public TrinoHealthStateType healthState()
{
return this.healthy;
return this.healthState;
}

public void healthy(boolean healthy)
public void healthState(TrinoHealthStateType healthState)
{
this.healthy = healthy;
this.healthState = healthState;
}

public String proxyTo()
Expand Down Expand Up @@ -187,7 +188,7 @@ private synchronized Optional<LocalStats> getClusterToRoute(String user, String
{
log.debug("sorting cluster stats for {} {}", user, routingGroup);
List<LocalStats> filteredList = clusterStats.stream()
.filter(stats -> stats.healthy())
.filter(stats -> stats.healthState() == TrinoHealthStateType.HEALTHY)
.filter(stats -> routingGroup.equals(stats.routingGroup()))
.collect(Collectors.toList());

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@
import com.google.common.cache.LoadingCache;
import io.airlift.log.Logger;
import io.trino.gateway.ha.clustermonitor.ClusterStats;
import io.trino.gateway.ha.clustermonitor.TrinoHealthStateType;
import io.trino.gateway.ha.config.ProxyBackendConfiguration;
import io.trino.gateway.proxyserver.ProxyServerConfiguration;
import jakarta.ws.rs.HttpMethod;
Expand Down Expand Up @@ -133,7 +134,7 @@ public void updateBackEndHealth(String backendId, Boolean value)
public void updateBackEndStats(List<ClusterStats> stats)
{
for (ClusterStats clusterStats : stats) {
updateBackEndHealth(clusterStats.clusterId(), clusterStats.healthy());
updateBackEndHealth(clusterStats.clusterId(), clusterStats.healthState() == TrinoHealthStateType.HEALTHY);
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,9 +15,12 @@

import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.ImmutableMap;
import com.google.inject.Injector;
import io.trino.gateway.baseapp.BaseApp;
import io.trino.gateway.ha.config.ProxyBackendConfiguration;
import io.trino.gateway.ha.router.GatewayCookie;
import io.trino.gateway.ha.router.OAuth2GatewayCookie;
import io.trino.gateway.ha.router.RoutingManager;
import okhttp3.Cookie;
import okhttp3.MediaType;
import okhttp3.OkHttpClient;
Expand All @@ -33,6 +36,7 @@
import org.testcontainers.containers.TrinoContainer;

import java.io.IOException;
import java.lang.reflect.Field;
import java.util.Base64;
import java.util.List;
import java.util.Optional;
Expand Down Expand Up @@ -85,7 +89,8 @@ public void setup()

// Start Gateway
String[] args = {"server", testConfig.configFilePath()};
HaGatewayLauncher.main(args);
HaGatewayLauncher haGatewayLauncher = new HaGatewayLauncher();
haGatewayLauncher.run(args);
// Now populate the backend
HaGatewayTestUtils.setUpBackend(
"trino1", "http://localhost:" + backend1Port, "externalUrl", true, "adhoc", routerPort);
Expand All @@ -95,6 +100,14 @@ public void setup()
HaGatewayTestUtils.setUpBackend(
"custom", "http://localhost:" + customBackendPort, "externalUrl", true, "custom",
routerPort);

// use injector to manually set backends to healthState
Field privateInjectorField = BaseApp.class.getDeclaredField("injector");
privateInjectorField.setAccessible(true);
Injector injector = (Injector) privateInjectorField.get(haGatewayLauncher);
injector.getInstance(RoutingManager.class).updateBackEndHealth("trino1", true);
injector.getInstance(RoutingManager.class).updateBackEndHealth("trino2", true);
injector.getInstance(RoutingManager.class).updateBackEndHealth("custom", true);
}

@Test
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -14,7 +14,10 @@
package io.trino.gateway.ha;

import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.inject.Injector;
import io.trino.gateway.baseapp.BaseApp;
import io.trino.gateway.ha.config.ProxyBackendConfiguration;
import io.trino.gateway.ha.router.RoutingManager;
import okhttp3.MediaType;
import okhttp3.OkHttpClient;
import okhttp3.Request;
Expand All @@ -27,6 +30,8 @@
import org.junit.jupiter.api.TestInstance.Lifecycle;
import org.testcontainers.containers.TrinoContainer;

import java.lang.reflect.Field;

import static org.assertj.core.api.Assertions.assertThat;

@TestInstance(Lifecycle.PER_CLASS)
Expand All @@ -50,10 +55,17 @@ public void setup()
HaGatewayTestUtils.buildGatewayConfigAndSeedDb(routerPort, "test-config-template.yml");
// Start Gateway
String[] args = {"server", testConfig.configFilePath()};
HaGatewayLauncher.main(args);
HaGatewayLauncher haGatewayLauncher = new HaGatewayLauncher();
haGatewayLauncher.run(args);
// Now populate the backend
HaGatewayTestUtils.setUpBackend(
"trino1", "http://localhost:" + backendPort, "externalUrl", true, "adhoc", routerPort);

// use injector to manually set backends to healthState
Field privateInjectorField = BaseApp.class.getDeclaredField("injector");
privateInjectorField.setAccessible(true);
Injector injector = (Injector) privateInjectorField.get(haGatewayLauncher);
injector.getInstance(RoutingManager.class).updateBackEndHealth("trino1", true);
}

@Test
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -74,6 +74,6 @@ private void testClusterStatsMonitor(Function<BackendStateConfiguration, Cluster

ClusterStats stats = monitor.monitor(proxyBackend);
assertThat(stats.clusterId()).isEqualTo("test_cluster");
assertThat(stats.healthy()).isTrue();
assertThat(stats.healthState()).isEqualTo(TrinoHealthStateType.HEALTHY);
}
}
Loading

0 comments on commit f1a2261

Please sign in to comment.