Skip to content
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

Handle unique query id on server #9648

Merged
merged 1 commit into from
Oct 26, 2022
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -179,17 +179,7 @@ public Map<Long, String> getRunningQueries() {
Set<ServerInstance> getRunningServers(long requestId) {
Preconditions.checkState(_queriesById != null, "Query cancellation is not enabled on broker");
QueryServers queryServers = _queriesById.get(requestId);
if (queryServers == null) {
return Collections.emptySet();
}
Set<ServerInstance> runningServers = new HashSet<>();
if (queryServers._offlineRoutingTable != null) {
runningServers.addAll(queryServers._offlineRoutingTable.keySet());
}
if (queryServers._realtimeRoutingTable != null) {
runningServers.addAll(queryServers._realtimeRoutingTable.keySet());
}
return runningServers;
return queryServers != null ? queryServers._servers : Collections.emptySet();
}

@Override
Expand All @@ -201,23 +191,12 @@ public boolean cancelQuery(long requestId, int timeoutMs, Executor executor, Htt
if (queryServers == null) {
return false;
}
// TODO: Use different global query id for OFFLINE and REALTIME table after releasing 0.12.0. See QueryIdUtils for
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Are we going to log different global query ids(or traceId as mentioned in this PR) for the same query in the broker log as well? That can help triage the queries in different components in the future.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

With this change, we will always log the same request id on both broker and servers. Query id and trace id are only available within the server, so you won't find them on the broker.

// details
String globalQueryId = getGlobalQueryId(requestId);
List<String> serverUrls = new ArrayList<>();
if (queryServers._offlineRoutingTable != null) {
for (ServerInstance server : queryServers._offlineRoutingTable.keySet()) {
serverUrls.add(String.format("%s/query/%s", server.getAdminEndpoint(), getGlobalQueryId(requestId)));
}
}
if (queryServers._realtimeRoutingTable != null) {
// NOTE: When the query is sent to both OFFLINE and REALTIME table, the REALTIME one has negative request id to
// differentiate from the OFFLINE one
long realtimeRequestId = queryServers._offlineRoutingTable == null ? requestId : -requestId;
for (ServerInstance server : queryServers._realtimeRoutingTable.keySet()) {
serverUrls.add(String.format("%s/query/%s", server.getAdminEndpoint(), getGlobalQueryId(realtimeRequestId)));
}
}
if (serverUrls.isEmpty()) {
LOGGER.debug("No servers running the query: {} right now", queryServers._query);
return true;
for (ServerInstance serverInstance : queryServers._servers) {
serverUrls.add(String.format("%s/query/%s", serverInstance.getAdminEndpoint(), globalQueryId));
}
LOGGER.debug("Cancelling the query: {} via server urls: {}", queryServers._query, serverUrls);
CompletionService<DeleteMethod> completionService =
Expand Down Expand Up @@ -1676,15 +1655,18 @@ public void setServerStats(String serverStats) {
* Helper class to track the query plaintext and the requested servers.
*/
private static class QueryServers {
private final String _query;
private final Map<ServerInstance, List<String>> _offlineRoutingTable;
private final Map<ServerInstance, List<String>> _realtimeRoutingTable;
final String _query;
final Set<ServerInstance> _servers = new HashSet<>();

public QueryServers(String query, @Nullable Map<ServerInstance, List<String>> offlineRoutingTable,
QueryServers(String query, @Nullable Map<ServerInstance, List<String>> offlineRoutingTable,
@Nullable Map<ServerInstance, List<String>> realtimeRoutingTable) {
_query = query;
_offlineRoutingTable = offlineRoutingTable;
_realtimeRoutingTable = realtimeRoutingTable;
if (offlineRoutingTable != null) {
_servers.addAll(offlineRoutingTable.keySet());
}
if (realtimeRoutingTable != null) {
_servers.addAll(realtimeRoutingTable.keySet());
}
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -100,10 +100,7 @@ protected BrokerResponseNative processBrokerRequest(long requestId, BrokerReques
}
if (realtimeBrokerRequest != null) {
assert realtimeRoutingTable != null;
// NOTE: When both OFFLINE and REALTIME request exist, use negative request id for REALTIME to differentiate
// from the OFFLINE one
long realtimeRequestId = offlineBrokerRequest == null ? requestId : -requestId;
sendRequest(realtimeRequestId, TableType.REALTIME, realtimeBrokerRequest, realtimeRoutingTable, responseMap,
sendRequest(requestId, TableType.REALTIME, realtimeBrokerRequest, realtimeRoutingTable, responseMap,
requestContext.isSampledRequest());
}
return _streamingReduceService.reduceOnStreamResponse(originalBrokerRequest, responseMap, timeoutMs,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -80,6 +80,7 @@
import org.apache.pinot.spi.exception.QueryCancelledException;
import org.apache.pinot.spi.trace.Tracing;
import org.apache.pinot.spi.utils.CommonConstants;
import org.apache.pinot.spi.utils.builder.TableNameBuilder;
import org.joda.time.Interval;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
Expand Down Expand Up @@ -135,7 +136,12 @@ public InstanceResponseBlock execute(ServerQueryRequest queryRequest, ExecutorSe
return executeInternal(queryRequest, executorService, responseObserver);
}
try {
Tracing.getTracer().register(queryRequest.getRequestId());
long requestId = queryRequest.getRequestId();
// NOTE: Use negative request id as trace id for REALTIME table to prevent id conflict when the same request
// hitting both OFFLINE and REALTIME table (hybrid table setup)
long traceId =
TableNameBuilder.isRealtimeTableResource(queryRequest.getTableNameWithType()) ? -requestId : requestId;
Tracing.getTracer().register(traceId);
return executeInternal(queryRequest, executorService, responseObserver);
} finally {
Tracing.getTracer().unregister();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,9 @@
import org.apache.pinot.core.query.request.context.QueryContext;
import org.apache.pinot.core.query.request.context.TimerContext;
import org.apache.pinot.core.query.request.context.utils.QueryContextConverterUtils;
import org.apache.pinot.core.query.utils.QueryIdUtils;
import org.apache.pinot.spi.utils.CommonConstants.Query.Request;
import org.apache.pinot.spi.utils.builder.TableNameBuilder;
import org.apache.pinot.sql.parsers.CalciteSqlCompiler;
import org.apache.thrift.TDeserializer;
import org.apache.thrift.protocol.TCompactProtocol;
Expand All @@ -51,6 +53,10 @@ public class ServerQueryRequest {
private final List<String> _segmentsToQuery;
private final QueryContext _queryContext;

// Request id might not be unique across brokers or for request hitting a hybrid table. To solve that we may construct
// a unique query id from broker id, request id and table type.
private final String _queryId;

// Timing information for different phases of query execution
private final TimerContext _timerContext;

Expand All @@ -61,6 +67,8 @@ public ServerQueryRequest(InstanceRequest instanceRequest, ServerMetrics serverM
_enableStreaming = false;
_segmentsToQuery = instanceRequest.getSearchSegments();
_queryContext = getQueryContext(instanceRequest.getQuery().getPinotQuery());
_queryId = QueryIdUtils.getQueryId(_brokerId, _requestId,
TableNameBuilder.getTableTypeFromTableName(_queryContext.getTableName()));
_timerContext = new TimerContext(_queryContext.getTableName(), serverMetrics, queryArrivalTimeMs);
}

Expand Down Expand Up @@ -88,6 +96,8 @@ public ServerQueryRequest(Server.ServerRequest serverRequest, ServerMetrics serv
throw new UnsupportedOperationException("Unsupported payloadType: " + payloadType);
}
_queryContext = getQueryContext(brokerRequest.getPinotQuery());
_queryId = QueryIdUtils.getQueryId(_brokerId, _requestId,
TableNameBuilder.getTableTypeFromTableName(_queryContext.getTableName()));
_timerContext = new TimerContext(_queryContext.getTableName(), serverMetrics, queryArrivalTimeMs);
}

Expand All @@ -100,14 +110,6 @@ private static QueryContext getQueryContext(PinotQuery pinotQuery) {
return queryContext;
}

/**
* As _requestId can be same across brokers, so use _brokerId and _requestId together to uniquely identify a query.
* @return unique query Id within a pinot cluster.
*/
public String getQueryId() {
return _brokerId + "_" + _requestId;
}

public long getRequestId() {
return _requestId;
}
Expand Down Expand Up @@ -136,6 +138,10 @@ public QueryContext getQueryContext() {
return _queryContext;
}

public String getQueryId() {
return _queryId;
}

public TimerContext getTimerContext() {
return _timerContext;
}
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,51 @@
/**
* 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.pinot.core.query.utils;

import org.apache.pinot.spi.config.table.TableType;


/**
* Utils to generate and manage the unique query id within a cluster.
* Request id might not be unique across brokers or for request hitting a hybrid table. To generate a unique query id
* within a cluster, we want to combine the broker id, request id and table type.
*/
public class QueryIdUtils {
private QueryIdUtils() {
}

private static final String OFFLINE_SUFFIX = "_O";
private static final String REALTIME_SUFFIX = "_R";

public static String getQueryId(String brokerId, long requestId, TableType tableType) {
return brokerId + "_" + requestId + (tableType == TableType.OFFLINE ? OFFLINE_SUFFIX : REALTIME_SUFFIX);
}

public static boolean hasTypeSuffix(String queryId) {
return queryId.endsWith(OFFLINE_SUFFIX) || queryId.endsWith(REALTIME_SUFFIX);
}

public static String withOfflineSuffix(String queryId) {
return queryId + OFFLINE_SUFFIX;
}

public static String withRealtimeSuffix(String queryId) {
return queryId + REALTIME_SUFFIX;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -106,14 +106,10 @@ public AsyncQueryResponse submitQuery(long requestId, String rawTableName,
}
if (realtimeBrokerRequest != null) {
assert realtimeRoutingTable != null;
// NOTE: When both OFFLINE and REALTIME request exist, use negative request id for REALTIME to differentiate
// from the OFFLINE one
long realtimeRequestId = offlineBrokerRequest == null ? requestId : -requestId;
for (Map.Entry<ServerInstance, List<String>> entry : realtimeRoutingTable.entrySet()) {
ServerRoutingInstance serverRoutingInstance =
entry.getKey().toServerRoutingInstance(TableType.REALTIME, preferTls);
InstanceRequest instanceRequest =
getInstanceRequest(realtimeRequestId, realtimeBrokerRequest, entry.getValue());
InstanceRequest instanceRequest = getInstanceRequest(requestId, realtimeBrokerRequest, entry.getValue());
requestMap.put(serverRoutingInstance, instanceRequest);
}
}
Expand Down Expand Up @@ -181,8 +177,7 @@ public void shutDown() {

void receiveDataTable(ServerRoutingInstance serverRoutingInstance, DataTable dataTable, int responseSize,
int deserializationTimeMs) {
// NOTE: For hybrid table, REALTIME request has negative request id
long requestId = Math.abs(Long.parseLong(dataTable.getMetadata().get(MetadataKey.REQUEST_ID.getName())));
long requestId = Long.parseLong(dataTable.getMetadata().get(MetadataKey.REQUEST_ID.getName()));
AsyncQueryResponse asyncQueryResponse = _asyncQueryResponseMap.get(requestId);

// Query future might be null if the query is already done (maybe due to failure)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -38,6 +38,8 @@
import javax.ws.rs.core.HttpHeaders;
import javax.ws.rs.core.MediaType;
import javax.ws.rs.core.Response;
import org.apache.pinot.core.query.utils.QueryIdUtils;
import org.apache.pinot.core.transport.InstanceRequestHandler;
import org.apache.pinot.server.starter.ServerInstance;

import static org.apache.pinot.spi.utils.CommonConstants.SWAGGER_AUTHORIZATION_KEY;
Expand Down Expand Up @@ -65,10 +67,20 @@ public class QueryResource {
@ApiResponse(code = 404, message = "Query not found running on the server")
})
public String cancelQuery(
@ApiParam(value = "QueryId as in the format of <brokerId>_<requestId>", required = true) @PathParam("queryId")
String queryId) {
@ApiParam(value = "QueryId as in the format of <brokerId>_<requestId> or <brokerId>_<requestId>_(O|R)",
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: it might make more sense for consistency with other APIs to make the offline/realtime part another optional API param. The default behavior has always been to cancel both, so I don't think there's backwards compatibility to worry about

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This API is on the server side, which is meant to be called by the broker. We need the type suffix to separate the OFFLINE and REALTIME query on the server side. Since they are tracked as 2 separate queries, it is better to give finer control.
On the broker side, the query cancellation will take the request id and always cancel both queries. When the server is upgraded, broker can cancel OFFLINE queries and REALTIME queries individually because they might fanout to different servers.

required = true)
@PathParam("queryId") String queryId) {
try {
if (_serverInstance.getInstanceRequestHandler().cancelQuery(queryId)) {
InstanceRequestHandler requestHandler = _serverInstance.getInstanceRequestHandler();
boolean queryCancelled;
if (QueryIdUtils.hasTypeSuffix(queryId)) {
queryCancelled = requestHandler.cancelQuery(queryId);
} else {
boolean offlineQueryCancelled = requestHandler.cancelQuery(QueryIdUtils.withOfflineSuffix(queryId));
boolean realtimeQueryCancelled = requestHandler.cancelQuery(QueryIdUtils.withRealtimeSuffix(queryId));
queryCancelled = offlineQueryCancelled | realtimeQueryCancelled;
}
if (queryCancelled) {
return "Cancelled query: " + queryId;
}
} catch (Exception e) {
Expand All @@ -85,7 +97,7 @@ public String cancelQuery(
@Path("/queries/id")
@Produces(MediaType.APPLICATION_JSON)
@ApiOperation(value = "Get queryIds of running queries on the server", notes = "QueryIds are in the format of "
+ "<brokerId>_<requestId>")
+ "<brokerId>_<requestId>_(O|R)")
@ApiResponses(value = {
@ApiResponse(code = 200, message = "Success"), @ApiResponse(code = 500, message = "Internal server error")
})
Expand Down