Skip to content

Commit

Permalink
add a new query laning metrics to visualize lane assignment (#12111)
Browse files Browse the repository at this point in the history
* add a new query laning metrics to visualize lane assignment

* fixes :spotbugs check

* Update docs/operations/metrics.md

Co-authored-by: Benedict Jin <[email protected]>

* Update server/src/main/java/org/apache/druid/server/QueryScheduler.java

Co-authored-by: Benedict Jin <[email protected]>

* Update server/src/main/java/org/apache/druid/server/QueryScheduler.java

Co-authored-by: Benedict Jin <[email protected]>

Co-authored-by: Benedict Jin <[email protected]>
  • Loading branch information
isandeep41 and asdf2014 authored Mar 4, 2022
1 parent 36bc418 commit 61e1ffc
Show file tree
Hide file tree
Showing 4 changed files with 41 additions and 4 deletions.
1 change: 1 addition & 0 deletions docs/operations/metrics.md
Original file line number Diff line number Diff line change
Expand Up @@ -58,6 +58,7 @@ Available Metrics
|`query/interrupted/count`|number of queries interrupted due to cancellation.|This metric is only available if the QueryCountStatsMonitor module is included.||
|`query/timeout/count`|number of timed out queries.|This metric is only available if the QueryCountStatsMonitor module is included.||
|`query/segments/count`|This metric is not enabled by default. See the `QueryMetrics` Interface for reference regarding enabling this metric. Number of segments that will be touched by the query. In the broker, it makes a plan to distribute the query to realtime tasks and historicals based on a snapshot of segment distribution state. If there are some segments moved after this snapshot is created, certain historicals and realtime tasks can report those segments as missing to the broker. The broker will re-send the query to the new servers that serve those segments after move. In this case, those segments can be counted more than once in this metric.|Varies.|
|`query/priority`|Assigned lane and priority, only if Laning strategy is enabled. Refer to [Laning strategies](../configuration/index.md#laning-strategies)|lane, dataSource, type|0|
|`sqlQuery/time`|Milliseconds taken to complete a SQL query.|id, nativeQueryIds, dataSource, remoteAddress, success.|< 1s|
|`sqlQuery/bytes`|number of bytes returned in SQL query response.|id, nativeQueryIds, dataSource, remoteAddress, success.| |

Expand Down
30 changes: 29 additions & 1 deletion server/src/main/java/org/apache/druid/server/QueryScheduler.java
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,10 @@
import org.apache.druid.java.util.common.concurrent.Execs;
import org.apache.druid.java.util.common.guava.LazySequence;
import org.apache.druid.java.util.common.guava.Sequence;
import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.java.util.emitter.core.NoopEmitter;
import org.apache.druid.java.util.emitter.service.ServiceEmitter;
import org.apache.druid.java.util.emitter.service.ServiceMetricEvent;
import org.apache.druid.query.Query;
import org.apache.druid.query.QueryCapacityExceededException;
import org.apache.druid.query.QueryContexts;
Expand All @@ -58,6 +62,7 @@
*/
public class QueryScheduler implements QueryWatcher
{
private static final Logger LOGGER = new Logger(QueryScheduler.class);
public static final int UNAVAILABLE = -1;
public static final String TOTAL = "total";
private final int totalCapacity;
Expand Down Expand Up @@ -86,12 +91,14 @@ public class QueryScheduler implements QueryWatcher
* but it is OK in most cases since they will be cleaned up once the query is done.
*/
private final SetMultimap<String, String> queryDatasources;
private final ServiceEmitter emitter;

public QueryScheduler(
int totalNumThreads,
QueryPrioritizationStrategy prioritizationStrategy,
QueryLaningStrategy laningStrategy,
ServerConfig serverConfig
ServerConfig serverConfig,
ServiceEmitter emitter
)
{
this.prioritizationStrategy = prioritizationStrategy;
Expand All @@ -108,6 +115,21 @@ public QueryScheduler(
this.totalCapacity = serverConfig.getNumThreads();
}
this.laneRegistry = BulkheadRegistry.of(getLaneConfigs(limitTotal));
this.emitter = emitter;
}

/**
* Keeping the old constructor as many test classes are dependent on this
*/
@VisibleForTesting
public QueryScheduler(
int totalNumThreads,
QueryPrioritizationStrategy prioritizationStrategy,
QueryLaningStrategy laningStrategy,
ServerConfig serverConfig
)
{
this(totalNumThreads, prioritizationStrategy, laningStrategy, serverConfig, new ServiceEmitter("test", "localhost", new NoopEmitter()));
}

@Override
Expand Down Expand Up @@ -137,6 +159,12 @@ public <T> Query<T> prioritizeAndLaneQuery(QueryPlus<T> queryPlus, Set<SegmentSe
Optional<Integer> priority = prioritizationStrategy.computePriority(queryPlus, segments);
query = priority.map(query::withPriority).orElse(query);
Optional<String> lane = laningStrategy.computeLane(queryPlus.withQuery(query), segments);
LOGGER.info("[%s] lane assigned to [%s] query with [%,d] priority", lane.orElse("default"), query.getType(), priority.orElse(Integer.valueOf(0)));
final ServiceMetricEvent.Builder builderUsr = ServiceMetricEvent.builder().setFeed("metrics")
.setDimension("lane", lane.orElse("default"))
.setDimension("dataSource", query.getDataSource().getTableNames())
.setDimension("type", query.getType());
emitter.emit(builderUsr.build("query/priority", priority.orElse(Integer.valueOf(0))));
return lane.map(query::withLane).orElse(query);
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,27 +23,30 @@
import com.fasterxml.jackson.annotation.JsonCreator;
import com.google.inject.Inject;
import com.google.inject.Provider;
import org.apache.druid.java.util.emitter.service.ServiceEmitter;
import org.apache.druid.server.initialization.ServerConfig;


public class QuerySchedulerProvider extends QuerySchedulerConfig implements Provider<QueryScheduler>
{
private final ServerConfig serverConfig;
private final ServiceEmitter emitter;

/**
* This needs to be both marked as guice injected to be bound correctly, and also marked with json creator and
* jackson inject to work with {@link org.apache.druid.guice.JsonConfigProvider}
*/
@Inject
@JsonCreator
public QuerySchedulerProvider(@JacksonInject ServerConfig serverConfig)
public QuerySchedulerProvider(@JacksonInject ServerConfig serverConfig, @JacksonInject ServiceEmitter emitter)
{
this.serverConfig = serverConfig;
this.emitter = emitter;
}

@Override
public QueryScheduler get()
{
return new QueryScheduler(getNumThreads(), getPrioritizationStrategy(), getLaningStrategy(), serverConfig);
return new QueryScheduler(getNumThreads(), getPrioritizationStrategy(), getLaningStrategy(), serverConfig, emitter);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -44,6 +44,8 @@
import org.apache.druid.java.util.common.guava.Sequences;
import org.apache.druid.java.util.common.guava.Yielder;
import org.apache.druid.java.util.common.guava.Yielders;
import org.apache.druid.java.util.emitter.core.NoopEmitter;
import org.apache.druid.java.util.emitter.service.ServiceEmitter;
import org.apache.druid.query.Query;
import org.apache.druid.query.QueryCapacityExceededException;
import org.apache.druid.query.QueryContexts;
Expand Down Expand Up @@ -697,13 +699,16 @@ private Injector createInjector()
ImmutableList.of(
binder -> {
binder.bind(ServerConfig.class).toInstance(new ServerConfig());
binder.bind(ServiceEmitter.class).toInstance(new ServiceEmitter("test", "localhost", new NoopEmitter()));
JsonConfigProvider.bind(binder, "druid.query.scheduler", QuerySchedulerProvider.class, Global.class);
}
)
);
ObjectMapper mapper = injector.getInstance(Key.get(ObjectMapper.class, Json.class));
mapper.setInjectableValues(
new InjectableValues.Std().addValue(ServerConfig.class, injector.getInstance(ServerConfig.class))
new InjectableValues.Std()
.addValue(ServerConfig.class, injector.getInstance(ServerConfig.class))
.addValue(ServiceEmitter.class, injector.getInstance(ServiceEmitter.class))
);
return injector;
}
Expand Down

0 comments on commit 61e1ffc

Please sign in to comment.