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

Add serverStats to BrokerQueryEventListener #14807

Open
wants to merge 2 commits into
base: master
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from 1 commit
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 @@ -859,7 +859,7 @@ protected BrokerResponse handleRequest(long requestId, String query, SqlNodeAndO
// Log query and stats
_queryLogger.log(
new QueryLogger.QueryLogParams(requestContext, tableName, brokerResponse, requesterIdentity, serverStats));

requestContext.setServerStatsMap(serverStats.getServerStatsMap());
return brokerResponse;
} finally {
Tracing.ThreadAccountantOps.clear();
Expand Down Expand Up @@ -1898,14 +1898,23 @@ private String getGlobalQueryId(long requestId) {
*/
public static class ServerStats {
private String _serverStats;
private Map<String, Map<String, Integer>> _serverStatsMap;

public String getServerStats() {
return _serverStats;
}

public Map getServerStatsMap() {
return _serverStatsMap;
}

public void setServerStats(String serverStats) {
_serverStats = serverStats;
}

public void setServerStatsMap(Map<String, Map<String, Integer>> serverStatsMap) {
_serverStatsMap = serverStatsMap;
}
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -126,6 +126,7 @@ protected BrokerResponseNative processBrokerRequest(long requestId, BrokerReques
System.nanoTime() - scatterGatherStartTimeNs);
// TODO Use scatterGatherStats as serverStats
serverStats.setServerStats(asyncQueryResponse.getServerStats());
serverStats.setServerStatsMap(asyncQueryResponse.getServerStatsMap());

int numServersQueried = finalResponses.size();
long totalResponseSize = 0;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicReference;
import java.util.stream.Collectors;
import javax.annotation.Nullable;
import javax.annotation.concurrent.ThreadSafe;
import org.apache.pinot.common.datatable.DataTable;
Expand Down Expand Up @@ -145,6 +146,21 @@ public String getServerStats() {
return stringBuilder.toString();
}

@Override
public Map<String, Map<String, Integer>> getServerStatsMap() {
Copy link
Contributor

Choose a reason for hiding this comment

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

can you add UTs for this?

return _responseMap.entrySet().stream()
.collect(Collectors.toMap(
entry -> entry.getKey().getShortName(),
Copy link
Contributor

Choose a reason for hiding this comment

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

The key here is the host right? is there a way to make it more explicit? Maybe instead of using Map<String, Map<String, Integer>> we can use Map<String, ServerStatsInfo> something and use POJO to easily extract the metric values.

entry -> Map.of(
"SubmitDelayMs", entry.getValue().getSubmitDelayMs(),
"ResponseDelayMs", entry.getValue().getResponseDelayMs(),
"ResponseSize", entry.getValue().getResponseSize(),
"DeserializationTimeMs", entry.getValue().getDeserializationTimeMs(),
"RequestSentDelayMs", entry.getValue().getRequestSentDelayMs()
)
));
}

@Override
public long getServerResponseDelayMs(ServerRoutingInstance serverRoutingInstance) {
return _responseMap.get(serverRoutingInstance).getResponseDelayMs();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -62,6 +62,8 @@ Map<ServerRoutingInstance, ServerResponse> getFinalResponses()
*/
String getServerStats();

Map<String, Map<String, Integer>> getServerStatsMap();

/**
* Returns the time taken for the server to respond to the query.
* @param serverRoutingInstance
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -81,6 +81,7 @@ public class DefaultRequestContext implements RequestScope {
private Map<String, String> _traceInfo = new HashMap<>();
private List<String> _processingExceptions = new ArrayList<>();
private Map<String, List<String>> _requestHttpHeaders = new HashMap<>();
private Map<String, Map<String, Integer>> _serverStatsMap = new HashMap<>();

public DefaultRequestContext() {
}
Expand Down Expand Up @@ -573,6 +574,16 @@ public void setRequestHttpHeaders(Map<String, List<String>> requestHttpHeaders)
_requestHttpHeaders.putAll(requestHttpHeaders);
}

@Override
public Map<String, Map<String, Integer>> getServerStatsMap() {
return _serverStatsMap;
}

@Override
public void setServerStatsMap(Map<String, Map<String, Integer>> serverStatsMap) {
_serverStatsMap.putAll(serverStatsMap);
Copy link
Contributor

Choose a reason for hiding this comment

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

this is counter-intuitive. Instead of set you can do addToServerStatsMap?

}

@Override
public void close() {
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -229,6 +229,10 @@ default boolean isSampledRequest() {
*/
void setRequestHttpHeaders(Map<String, List<String>> requestHttpHeaders);

Map<String, Map<String, Integer>> getServerStatsMap();

void setServerStatsMap(Map<String, Map<String, Integer>> serverStatsMap);

enum FanoutType {
OFFLINE, REALTIME, HYBRID
}
Expand Down
Loading