-
Notifications
You must be signed in to change notification settings - Fork 80
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Configure the router policy via router provider module #98
Conversation
vishalya
commented
Nov 24, 2023
•
edited
Loading
edited
- This PR providers a way to configure the que length based router.
- This router and it's logic was there, now it can be configured via the config file.
- Please check the configuration.md, Router Policy section to understand on how to configure the current default router vs the queue length based router
- HaGatewayProviderModule needed to be refactored and the db dependencies are moved into another base module, because of that the router module needs to be configured explicitly.
- Hopefully this PR fixes Feature Request: Routing based on number of queued and running queries #77 and Load balancing scheme in Trino gateway? #84
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Looks good overall. My only real concern is that as config gets more complex, we should make sure we're giving understandable error messages to users and avoiding returning hundreds of lines of Guice injection errors.
docs/configuration.md
Outdated
There are 2 routers you can choose from | ||
### Que length based | ||
- This distributed the queries based on the number of queries running |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
suggestion: Queries are routed to the backend with the fewest queued queries. If multiple backends have zero queued queries, the backend with the fewest running queries is chosen.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yes, it should route it to the cluster with the fewest running queries.
gateway-ha/gateway-ha-config.yml
Outdated
- io.trino.gateway.ha.module.QueueLengthListenerModule | ||
- io.trino.gateway.ha.module.QueueLengthRouterProvider |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The QueueLengthRouterProvider
requires that the QueueLengthListenerModule
is configured (right?). If it isn't, what kind of error does the end user see?
If its not an easy one to understand, we should add some checking in Baseapp to ensure that dependent modules are loaded. AppConfiguration.getModules()
should give us the list of loaded modules.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
In my latest change, I am providing the BasicRouter by default and config can override it. Is that ok?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
sgtm
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
done.
This needs a rebase before we can look at it again. |
Yes, let me rebase and then I can also add a new router which is much simpler. Currently, we are testing it internally. |
9c5bea3
to
41fa781
Compare
The logic for the |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
A few things to address, mainly around the locking and potential performance impacts
// The live stats refresh every few seconds, so we update the stats immediately | ||
// so that they can be used for next queries to route | ||
// We assume that if a user has queued queries then newly arriving queries | ||
// for that user would also be queued |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
We assume that if a user has queued queries then newly arriving queries
This is a strong assumption given trino resource group options such as queryType
and queryText
.
My main concern however is putting a synchronized method in the getBackend
critical path. This could cause performance issues at high concurrency.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
- I hear you on the assumption, but one could only assume in absense of the live stats. Currently, the cluster stats interval can be set with a granularity of seconds (default is 60 secs), but we are doing ok with 10 sec interval. This assumption phase is applicable during that 10 seconds and when the fresh stats arrive we start afresh with them.
- There is tremendous overall gain in the performance, the BasicRouter (the one on the main branch) checks db for every request and it's literally 50 to 100 times slower. I have verified that.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
You mean this is faster compared to the TrinoQueueLengthRoutingTable
I assume?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Even compared to the BasicRouter it's very fast, since BasicRouter keeps cheking db for the active/inactive state.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This synchronized usage is incorrect.
Reads and writes on userQueuedCount
and runningQueryCount
must be synchronized.
Change from record class to normal class requires a lot of works on reimplementing getter/setters/equals. I think we can continue to use record class by using AtomicInteger and ConcurrentHashMap.
import io.trino.gateway.ha.router.HaRoutingManager; | ||
import io.trino.gateway.ha.router.RoutingManager; | ||
|
||
public class BasicRouterProvider |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The class name should reflect the router it provides, so HaRouterProvider
. Tbh the HaRoutingManager
could have a more meaningful name such as StochasticRoutingManager
, this would be a good time to update it.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
so StochasticRoutingManager
and StochasticRoutingManagerProvide
? I am ok with that.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
👍
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
done
Lock readLock = lock.readLock(); | ||
try { | ||
readLock.lock(); | ||
filteredList = clusterStats.stream() |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This would be a good use case for a Map<Array<ClusterStats>>
or potentially com.google.common.collect.Table
if you need to look up on various attributes vs filtering a list
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yeah, If it becomes more complicated then yes, may be the next iteration when we get more stats for the cost based routing we can use it. At this time the filter and sort is just simple and clean IMO.
gateway-ha/src/main/java/io/trino/gateway/ha/router/QueryCountBasedRouter.java
Outdated
Show resolved
Hide resolved
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 would greatly enhance our current setup and would be great feature for the community. |
e3501bf
to
819af8f
Compare
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Getting close, just spotted a few more code improvements to make.
Can you additionally remove TrinoQueueLengthRoutingTable
as part of this PR? It will be orphaned once this merged.
|
||
private void validateModules(List<AppModule> modules, T configuration, Environment environment) | ||
{ | ||
Optional routerProvider = modules.stream() |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Add a <TypeParameter>
to this declaration
ClusterStats monitor(ProxyBackendConfiguration backend); | ||
abstract ClusterStats monitor(ProxyBackendConfiguration backend); | ||
|
||
protected void populateClusterStats(ClusterStats.Builder stats, ProxyBackendConfiguration backend) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This should return a ClusterStatsBuilder instead of modifying one passed as an argument. It looks like the builder passed is initialized with ClusterStats.builder(backend.getName())
, so just change this to
ClusterStats.Builder getClusterStatsBuilder(ProxyBackendConfiguration backend)
{
ClusterStats.Builder builder = ProxyBackendConfiguration backend);
//...
return builder;
}
import io.trino.gateway.ha.router.HaRoutingManager; | ||
import io.trino.gateway.ha.router.RoutingManager; | ||
|
||
public class BasicRouterProvider |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
👍
return Optional.empty(); | ||
} | ||
|
||
Collections.sort(filteredList, new Comparator<ClusterStats>() |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
replace with Collections.max()
instead of sort and get 0
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
It would be Collections.min()
but I got the idea.
|
||
Collections.sort(filteredList, new Comparator<ClusterStats>() | ||
{ | ||
public int compare(ClusterStats lhs, ClusterStats rhs) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
ClusterStats
should extend Comparable
, then you can implement compareTo
using this logic. That way it can be reused in other routers.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
That's true! The comparison involves a user specific stats, so it won't be straight forward comparison, but I can a compare function and that can be resused and also makes the router code more readable with a simple lambda.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
looks good, can you remove the comment block of the previous code?
{ | ||
Optional<ClusterStats> cluster = getClusterToRoute(user, routingGroup); | ||
if (cluster.isPresent()) { | ||
cluster.orElseThrow().updateLocalStats(user); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Why is orElseThrow
used? isPresent()
was just checked. If this is needed then an exception supplier should be passed.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
That's because the static check doesn't allow it, but I guess there should be a way around it.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
you should be able to eliminate the isPresent
condition if you use ifPresent
for the updateLocalStats
call and return a flatMap
from cluster
cab9b71
to
c39d411
Compare
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Just a few loose ends
{ | ||
Optional<ClusterStats> cluster = getClusterToRoute(user, routingGroup); | ||
if (cluster.isPresent()) { | ||
cluster.orElseThrow().updateLocalStats(user); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
you should be able to eliminate the isPresent
condition if you use ifPresent
for the updateLocalStats
call and return a flatMap
from cluster
.findFirst(); | ||
if (routerProvider.isEmpty()) { | ||
logger.warn("Router provider doesn't exist in the config, using the StochasticRoutingManagerProvider"); | ||
String clazz = "io.trino.gateway.ha.module.StochasticRoutingManagerProvider"; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
this hard coded name could be missed in a refactor - use StochasticRoutingManagerProvider.class.getCanonicalName()
|
||
Collections.sort(filteredList, new Comparator<ClusterStats>() | ||
{ | ||
public int compare(ClusterStats lhs, ClusterStats rhs) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
looks good, can you remove the comment block of the previous code?
Optional<String> proxyUrl = getBackendForRoutingGroup(routingGroup, user); | ||
if (proxyUrl.isPresent()) { | ||
return proxyUrl.orElseThrow(); | ||
} | ||
return provideAdhocBackend(user); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Optional<String> proxyUrl = getBackendForRoutingGroup(routingGroup, user); | |
if (proxyUrl.isPresent()) { | |
return proxyUrl.orElseThrow(); | |
} | |
return provideAdhocBackend(user); | |
return getBackendForRoutingGroup(routingGroup, user).OrElse(provideAdhocBackend(user)); |
c39d411
to
067f2d8
Compare
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Just skimmed.
gateway-ha/src/main/java/io/trino/gateway/ha/clustermonitor/ClusterStats.java
Outdated
Show resolved
Hide resolved
gateway-ha/src/test/java/io/trino/gateway/ha/router/TestQueryCountBasedRouter.java
Outdated
Show resolved
Hide resolved
gateway-ha/src/test/java/io/trino/gateway/ha/router/TestQueryCountBasedRouter.java
Outdated
Show resolved
Hide resolved
gateway-ha/src/main/java/io/trino/gateway/ha/router/QueryCountBasedRouter.java
Show resolved
Hide resolved
gateway-ha/src/main/java/io/trino/gateway/ha/router/QueryCountBasedRouter.java
Outdated
Show resolved
Hide resolved
gateway-ha/src/main/java/io/trino/gateway/ha/clustermonitor/ClusterStats.java
Outdated
Show resolved
Hide resolved
067f2d8
to
288cdce
Compare
51997af
to
9fbc3e7
Compare
I have taken care of the review comments, the rebase is coming next. |
@ebyhr please take a look when you get a chance, changes and rebase is done. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Some comments on the implementation of concurrency.
// The live stats refresh every few seconds, so we update the stats immediately | ||
// so that they can be used for next queries to route | ||
// We assume that if a user has queued queries then newly arriving queries | ||
// for that user would also be queued |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This synchronized usage is incorrect.
Reads and writes on userQueuedCount
and runningQueryCount
must be synchronized.
Change from record class to normal class requires a lot of works on reimplementing getter/setters/equals. I think we can continue to use record class by using AtomicInteger and ConcurrentHashMap.
private ArrayList<ClusterStats> clusterStats; | ||
private static final ReentrantReadWriteLock lock = new ReentrantReadWriteLock(true); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The operations on clusterStats
are read all and replace all.
ReentrantReadWriteLock
can be replaced by AtomicReference<List<ClusterStats>>
.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Good catch on the read, thanks!
Both suggestions for the synchronization are good.
I might need to modify the flow a bit to use the record, with record I won't be able to modify the member attributes.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
-
I am going back to the class for
ClusterStats
as the router needs to update it, it can be achieved by copying the records back and forth, but it won't be performant. -
I have moved the reading/sorting and updating of the cluster stats at the router level - inside
QueryCountBasedRouter
, for 2 reasons.-
The logic is specific to the router.
-
The logic needs to read the stats, sort and the update the counters in the
ClusterStats
in a single transaction. This would avoid the race condition while processing simultaneous requests.
-
Thanks for the refactoring. The logic is easier to follow now. If I understand correctly,
We can keep
Updates: Discussed the following in contributor meeting. The current implementation works well in production.
|
I have pushed the changes to copy the stats locally and the router now works with them as discussed. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Just some minor comments. Others LGTM.
I think coding style can be improved overtime.
Hope we can get this merged soon to avoid conflicts on migration to JDBI & Airlift.
return clusterStats; | ||
} | ||
|
||
class LocalStats |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Add static
List<LocalStats> clusterStats() | ||
{ | ||
return clusterStats; | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Let's prevent someone misuse this function in the future.
List<LocalStats> clusterStats() | |
{ | |
return clusterStats; | |
} | |
@VisibleForTesting | |
synchronized List<LocalStats> clusterStats() | |
{ | |
return new ArrayList<>(clusterStats); | |
} |
return Optional.of(Collections.min(filteredList, (lhs, rhs) -> compareStats(lhs, rhs, user))); | ||
} | ||
|
||
private void updateLocalStats(LocalStats stats, String user) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Add synchronized
.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Looks good. As agreed with @willmostly @Chaho12 and others in the dev sync, we are merging this now and will address any code style and other minor issues in follow up PRs.