trinodb / trino-gateway

Home Page:https://trinodb.github.io/trino-gateway/

Geek Repo:Geek Repo

Github PK Tool:Github PK Tool

Feature Request: Routing based on number of queued and running queries

alaturqua opened this issue · comments

Description

Trino Gateway routes queries randomly within a routing group and does not consider the number of already running and queued queries on a cluster within a routing group

Issue

It can happen that some queries take longer and and queue of the routed cluster keeps growing

Example scenario:

Assumption 3 Clusters in a routing group.

  • Each cluster receive 50 queries
  • cluster 1 and 2 process all the queries and have 0 queries remaining in queued queries
  • cluster 3 has received some complex queries and processing needs longer and blocks other queries
  • Gateway routes randomly again additional 50 queries to each cluster
  • all clusters receive 50 queries to process again
  • cluster 3 has 100 queries now, because the one or couple complex queries are in running state and blocking others.

image
image

Feature Request

Implement a solution which routes queries based on health status of the backend. It should consider the number of running and queued queries and route new queries to clusters with less load

Findings on current code

There is already code for this, but it is not used as of now. ClusterStateListenerModule does not have code for it.
https://github.com/trinodb/trino-gateway/blob/main/gateway-ha/src/main/java/io/trino/gateway/ha/router/TrinoQueueLengthRoutingTable.java
https://github.com/trinodb/trino-gateway/blob/main/gateway-ha/src/main/java/io/trino/gateway/ha/clustermonitor/TrinoQueueLengthChecker.java

I can send a PR to try it out, we will need to test out thoroughly.

I can test it in our dev environment with realistic scenarios.

I ran some tests on the fork/PR in our dev environment.

Tests were done in a realistic scenario using 3x Clusters as trino backend with a routing group named looker. Routing rules make sure that every query with source = 'Looker' is routed to routing group looker.

---
name: "Looker"
description: "Route Looker queries to Looker"
priority: 6
condition: "request.getHeader(\"X-Trino-Source\") contains \"Looker\""
actions:
  - "result.put(\"routingGroup\", \"looker\")"

We have set the configs as follows:

routingRules:
  rulesEngineEnabled: true
  rulesConfigPath: <path-to-config>

requestRouter:
  port: 8080
  name: trinoRouter
  historySize: 2000

dataStore:
  jdbcUrl: jdbc:postgresql://DB_HOST:DB_PORT/DB_SCHEMA
  user: DB_USER
  password: DB_PASSWORD
  driver: org.postgresql.Driver
  queryHistoryHoursRetention: 24

# dataStore:
#   jdbcUrl: jdbc:mysql://MYSQL_HOST:MYSQL_PORT/MYSQL_SCHEMA
#   user: MYSQL_USER
#   password: MYSQL_PASSWORD
#   driver: com.mysql.cj.jdbc.Driver
#   queryHistoryHoursRetention: 24


backendState:
  username: <TRINO_USERNAME>
  password: <TRINO_PASSWORD>
  ssl: true

clusterStatsConfiguration:
  useApi: false

extraWhitelistPaths:
  - "/ui/insights"
  - "/api/v1/biac"
  - "/api/v1/dataProduct"
  - "/api/v1/dataproduct"
  - "/ext/faster"
  - "/oidc/callback"

server:
  applicationConnectors:
    - type: http
      port: 8090
  adminConnectors:
    - type: http
      port: 8091

# This can be adjusted based on the coordinator state
monitor:
  connectionTimeout: 600
  taskDelayMin: 0.1

modules:
  - io.trino.gateway.ha.module.HaGatewayProviderModule
  - io.trino.gateway.ha.module.ClusterStatsMonitorModule
  - io.trino.gateway.ha.module.QueueLengthListenerModule
  - io.trino.gateway.ha.module.QueueLengthRouterProvider

managedApps:
  - io.trino.gateway.ha.GatewayManagedApp
  - io.trino.gateway.ha.clustermonitor.ActiveClusterMonitor

# Logging settings.
logging:
  # The default level of all loggers. Can be OFF, ERROR, WARN, INFO, DEBUG, TRACE, or ALL.
  level: INFO

  # Logger-specific levels.
  loggers:
    io.trino: INFO

  appenders:
    - type: console
      filterFactories:
        - type: Log-filter-factory

Unfortunately the behavior did not change. We had the same issue like using Random Routing.
As you can see in the screenshot one cluster receives new requests although the queue has already 100 queries. And the others have.

Expectation would be that the requests/queries are distributed within a routing group almost equally.

image

Exactly what I'm looking for ;) If any help is needed do not. Il will see if I can reproduce the use case and make it works.

Worked today on resolving the issues alongside @vishalya. The solution now operates as intended, effectively distributing queries based on the number of queued queries evenly across a routing group. A merge and release would greatly enhance our current setup and would be great feature for the community.

#98

Once the rebase of the PR is in I can review and we can look at merging. Might not make it for version 6 but we will cut releases more often as well.