ccaominh commented on a change in pull request #9407: query laning and load shedding URL: https://github.com/apache/druid/pull/9407#discussion_r388631851
########## File path: server/src/main/java/org/apache/druid/server/QueryScheduler.java ########## @@ -0,0 +1,226 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.druid.server; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.HashMultimap; +import com.google.common.collect.Multimaps; +import com.google.common.collect.SetMultimap; +import com.google.common.util.concurrent.ListenableFuture; +import io.github.resilience4j.bulkhead.Bulkhead; +import io.github.resilience4j.bulkhead.BulkheadConfig; +import io.github.resilience4j.bulkhead.BulkheadRegistry; +import it.unimi.dsi.fastutil.objects.Object2IntMap; +import org.apache.druid.client.SegmentServerSelector; +import org.apache.druid.java.util.common.concurrent.Execs; +import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.query.Query; +import org.apache.druid.query.QueryContexts; +import org.apache.druid.query.QueryPlus; +import org.apache.druid.query.QueryWatcher; + +import java.time.Duration; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; + +/** + * QueryScheduler (potentially) assigns any {@link Query} that is to be executed to a 'query lane' using the + * {@link QueryLaningStrategy} that is defined in {@link QuerySchedulerConfig}. + * + * As a {@link QueryWatcher}, it also provides cancellation facilities. + * + * This class is shared by all requests on the Jetty HTTP theadpool and must be thread safe. + */ +public class QueryScheduler implements QueryWatcher +{ + static final String TOTAL = "default"; + private final QueryLaningStrategy laningStrategy; + private final BulkheadRegistry laneRegistry; + private final SetMultimap<String, ListenableFuture<?>> queryFutures; + private final SetMultimap<String, String> queryDatasources; + + public QueryScheduler(int totalNumThreads, QueryLaningStrategy laningStrategy) + { + this.laningStrategy = laningStrategy; + this.laneRegistry = BulkheadRegistry.of(getLaneConfigs(totalNumThreads)); + this.queryFutures = Multimaps.synchronizedSetMultimap(HashMultimap.create()); + this.queryDatasources = Multimaps.synchronizedSetMultimap(HashMultimap.create()); + } + + @Override + public void registerQueryFuture(Query<?> query, ListenableFuture<?> future) + { + final String id = query.getId(); + final Set<String> datasources = query.getDataSource().getTableNames(); + queryFutures.put(id, future); + queryDatasources.putAll(id, datasources); + future.addListener( + () -> { + queryFutures.remove(id, future); + for (String datasource : datasources) { + queryDatasources.remove(id, datasource); + } + }, + Execs.directExecutor() + ); + } + + /** + * Assign a query a lane (if not set) + */ + public <T> Query<T> laneQuery(QueryPlus<T> queryPlus, Set<SegmentServerSelector> segments) + { + Query<T> query = queryPlus.getQuery(); + // man wins over machine.. for now. + if (QueryContexts.getLane(query) != null) { + return query; + } + Optional<String> lane = laningStrategy.computeLane(queryPlus, segments); + return lane.map(query::withLane).orElse(query); + } + + /** + * Run a query with the scheduler, attempting to acquire a semaphore from the total and lane specific query capacities + * + * Note that {@link #cancelQuery} should not interrupt the thread that calls run, in all current usages it only + * cancels any {@link ListenableFuture} created downstream. If this ever commonly changes, we should add + * synchronization between {@link #cancelQuery} and the acquisition of the {@link Bulkhead} to continue to ensure that + * anything acquired is also released. + * + * In the meantime, if a {@link ListenableFuture} is registered for the query that calls this method, it MUST handle + * this synchronization itself to ensure that no {@link Bulkhead} is acquired without releasing it. + */ + public <T> Sequence<T> run(Query<?> query, Sequence<T> resultSequence) + { + List<Bulkhead> bulkheads = acquireLanes(query); + return resultSequence.withBaggage(() -> releaseLanes(bulkheads)); + } + + /** + * Forcibly cancel all futures that have been registered to a specific query id + */ Review comment: Thanks for adding all the javadocs that were missing before! ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: [email protected] With regards, Apache Git Services --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
