From 2b339afb57c0f990136d7bf49a526f30795b3ca8 Mon Sep 17 00:00:00 2001 From: egalpin Date: Fri, 31 May 2024 17:20:49 -0700 Subject: [PATCH 01/49] Allows multiple requests per server per request ID --- .../BaseSingleStageBrokerRequestHandler.java | 123 ++++---- .../GrpcBrokerRequestHandler.java | 48 ++-- .../SingleConnectionBrokerRequestHandler.java | 36 +-- .../pinot/common/datatable/DataTable.java | 5 +- .../common/datatable/DataTableImplV4.java | 1 + pinot-common/src/thrift/request.thrift | 1 + .../reader/PinotServerDataFetcher.scala | 12 +- .../executor/ServerQueryExecutorV1Impl.java | 30 +- .../reduce/AggregationDataTableReducer.java | 6 +- .../query/reduce/BrokerReduceService.java | 61 ++-- .../core/query/reduce/DataTableReducer.java | 15 +- .../reduce/DistinctDataTableReducer.java | 16 +- .../reduce/ExecutionStatsAggregator.java | 8 +- .../reduce/ExplainPlanDataTableReducer.java | 13 +- .../query/reduce/GroupByDataTableReducer.java | 4 +- .../reduce/SelectionDataTableReducer.java | 7 +- .../query/request/ServerQueryRequest.java | 9 + .../core/query/scheduler/QueryScheduler.java | 2 + .../core/transport/AsyncQueryResponse.java | 109 +++++--- .../core/transport/DirectOOMHandler.java | 3 +- .../transport/InstanceRequestHandler.java | 9 +- .../pinot/core/transport/QueryResponse.java | 4 +- .../pinot/core/transport/QueryRouter.java | 104 ++++--- .../pinot/core/transport/ServerChannels.java | 50 ++-- .../pinot/core/transport/ServerInstance.java | 19 +- .../transport/ServerQueryRoutingContext.java | 69 +++++ .../core/transport/ServerRoutingInstance.java | 46 ++- .../query/reduce/BrokerReduceServiceTest.java | 9 +- .../reduce/StreamingReduceServiceTest.java | 5 +- .../core/transport/QueryRoutingTest.java | 262 ++++++++++++------ .../core/transport/ServerChannelsTest.java | 3 +- .../apache/pinot/queries/BaseQueriesTest.java | 24 +- .../pinot/queries/ExplainPlanQueriesTest.java | 13 +- 33 files changed, 670 insertions(+), 456 deletions(-) create mode 100644 pinot-core/src/main/java/org/apache/pinot/core/transport/ServerQueryRoutingContext.java diff --git a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BaseSingleStageBrokerRequestHandler.java b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BaseSingleStageBrokerRequestHandler.java index 6a839d5e36b..a09a3f43990 100644 --- a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BaseSingleStageBrokerRequestHandler.java +++ b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BaseSingleStageBrokerRequestHandler.java @@ -72,6 +72,7 @@ import org.apache.pinot.common.request.Identifier; import org.apache.pinot.common.request.Literal; import org.apache.pinot.common.request.PinotQuery; +import org.apache.pinot.core.transport.ServerQueryRoutingContext; import org.apache.pinot.common.response.BrokerResponse; import org.apache.pinot.common.response.ProcessingException; import org.apache.pinot.common.response.broker.BrokerResponseNative; @@ -620,44 +621,30 @@ protected BrokerResponse handleRequest(long requestId, String query, @Nullable S // Calculate routing table for the query // TODO: Modify RoutingManager interface to directly take PinotQuery long routingStartTimeNs = System.nanoTime(); - Map, List>> offlineRoutingTable = null; - Map, List>> realtimeRoutingTable = null; + Map> queryRoutingTable = null; List unavailableSegments = new ArrayList<>(); int numPrunedSegmentsTotal = 0; + if (offlineBrokerRequest != null) { - // NOTE: Routing table might be null if table is just removed - RoutingTable routingTable = _routingManager.getRoutingTable(offlineBrokerRequest, requestId); - if (routingTable != null) { - unavailableSegments.addAll(routingTable.getUnavailableSegments()); - Map, List>> serverInstanceToSegmentsMap = - routingTable.getServerInstanceToSegmentsMap(); - if (!serverInstanceToSegmentsMap.isEmpty()) { - offlineRoutingTable = serverInstanceToSegmentsMap; - } else { - offlineBrokerRequest = null; - } - numPrunedSegmentsTotal += routingTable.getNumPrunedSegments(); - } else { + Integer numPrunedSegments = + updateRoutingTable(requestId, offlineBrokerRequest, queryRoutingTable, unavailableSegments); + if (numPrunedSegments == null) { offlineBrokerRequest = null; + } else { + numPrunedSegmentsTotal += numPrunedSegments; } } + if (realtimeBrokerRequest != null) { - // NOTE: Routing table might be null if table is just removed - RoutingTable routingTable = _routingManager.getRoutingTable(realtimeBrokerRequest, requestId); - if (routingTable != null) { - unavailableSegments.addAll(routingTable.getUnavailableSegments()); - Map, List>> serverInstanceToSegmentsMap = - routingTable.getServerInstanceToSegmentsMap(); - if (!serverInstanceToSegmentsMap.isEmpty()) { - realtimeRoutingTable = serverInstanceToSegmentsMap; - } else { - realtimeBrokerRequest = null; - } - numPrunedSegmentsTotal += routingTable.getNumPrunedSegments(); - } else { + Integer numPrunedSegments = + updateRoutingTable(requestId, realtimeBrokerRequest, queryRoutingTable, unavailableSegments); + if (numPrunedSegments == null) { realtimeBrokerRequest = null; + } else { + numPrunedSegmentsTotal += numPrunedSegments; } } + int numUnavailableSegments = unavailableSegments.size(); requestContext.setNumUnavailableSegments(numUnavailableSegments); @@ -717,13 +704,8 @@ protected BrokerResponse handleRequest(long requestId, String query, @Nullable S // Set the maximum serialized response size per server, and ask server to directly return final response when only // one server is queried - int numServers = 0; - if (offlineRoutingTable != null) { - numServers += offlineRoutingTable.size(); - } - if (realtimeRoutingTable != null) { - numServers += realtimeRoutingTable.size(); - } + int numServers = queryRoutingTable.size(); + if (offlineBrokerRequest != null) { Map queryOptions = offlineBrokerRequest.getPinotQuery().getQueryOptions(); setMaxServerResponseSizeBytes(numServers, queryOptions, offlineTableConfig); @@ -762,11 +744,11 @@ protected BrokerResponse handleRequest(long requestId, String query, @Nullable S if (pinotQuery.isExplain()) { // Update routing tables to only send request to offline servers for OFFLINE and HYBRID tables. // TODO: Assess if the Explain Plan Query should also be routed to REALTIME servers for HYBRID tables - if (offlineRoutingTable != null) { - // For OFFLINE and HYBRID tables, don't send EXPLAIN query to realtime servers. - realtimeBrokerRequest = null; - realtimeRoutingTable = null; - } +// if (offlineRoutingTable != null) { +// // For OFFLINE and HYBRID tables, don't send EXPLAIN query to realtime servers. +// realtimeBrokerRequest = null; +// realtimeRoutingTable = null; +// } } BrokerResponseNative brokerResponse; if (_queriesById != null) { @@ -778,20 +760,20 @@ protected BrokerResponse handleRequest(long requestId, String query, @Nullable S // can always list the running queries and cancel query again until it ends. Just that such race // condition makes cancel API less reliable. This should be rare as it assumes sending queries out to // servers takes time, but will address later if needed. - _queriesById.put(requestId, new QueryServers(query, offlineRoutingTable, realtimeRoutingTable)); + _queriesById.put(requestId, new QueryServers(query, queryRoutingTable)); LOGGER.debug("Keep track of running query: {}", requestId); try { - brokerResponse = processBrokerRequest(requestId, brokerRequest, serverBrokerRequest, offlineBrokerRequest, - offlineRoutingTable, realtimeBrokerRequest, realtimeRoutingTable, remainingTimeMs, serverStats, - requestContext); + brokerResponse = + processBrokerRequest(requestId, brokerRequest, serverBrokerRequest, queryRoutingTable, remainingTimeMs, + serverStats, requestContext); } finally { _queriesById.remove(requestId); LOGGER.debug("Remove track of running query: {}", requestId); } } else { - brokerResponse = processBrokerRequest(requestId, brokerRequest, serverBrokerRequest, offlineBrokerRequest, - offlineRoutingTable, realtimeBrokerRequest, realtimeRoutingTable, remainingTimeMs, serverStats, - requestContext); + brokerResponse = + processBrokerRequest(requestId, brokerRequest, serverBrokerRequest, queryRoutingTable, remainingTimeMs, + serverStats, requestContext); } for (ProcessingException exception : exceptions) { @@ -828,6 +810,33 @@ protected BrokerResponse handleRequest(long requestId, String query, @Nullable S } } + private Integer updateRoutingTable(long requestId, BrokerRequest brokerRequest, + Map> routingTableResult, List unavailableSegments) { + // NOTE: Routing table might be null if table is just removed + RoutingTable routingTable = _routingManager.getRoutingTable(brokerRequest, requestId); + if (routingTable != null) { + unavailableSegments.addAll(routingTable.getUnavailableSegments()); + Map, List>> serverInstanceToSegmentsMap = + routingTable.getServerInstanceToSegmentsMap(); + if (!serverInstanceToSegmentsMap.isEmpty()) { + for (Map.Entry, List>> serverMap + : serverInstanceToSegmentsMap.entrySet()) { + ServerInstance serverInstance = serverMap.getKey(); + Pair, List> segmentsToQuery = serverMap.getValue(); + + routingTableResult.computeIfAbsent(serverInstance, k -> new ArrayList<>()).add( + new ServerQueryRoutingContext(brokerRequest, segmentsToQuery, + serverInstance.toServerRoutingInstance(serverInstance.isTlsEnabled()))); + } + } else { + return null; + } + return routingTable.getNumPrunedSegments(); + } else { + return null; + } + } + private BrokerResponseNative getEmptyBrokerOnlyResponse(PinotQuery pinotQuery, RequestContext requestContext, String tableName, @Nullable RequesterIdentity requesterIdentity) { if (pinotQuery.isExplain()) { @@ -879,7 +888,7 @@ private void setTimestampIndexExpressionOverrideHints(@Nullable Expression expre case "datetrunc": String granularString = function.getOperands().get(0).getLiteral().getStringValue().toUpperCase(); Expression timeExpression = function.getOperands().get(1); - if (((function.getOperandsSize() == 2) || (function.getOperandsSize() == 3 && "MILLISECONDS".equalsIgnoreCase( + if (((function.getOperandsSize() == 2) || (function.getOperandsSize() == 3 && "MILLISECONDS" .equalsIgnoreCase( function.getOperands().get(2).getLiteral().getStringValue()))) && TimestampIndexUtils.isValidGranularity( granularString) && timeExpression.getIdentifier() != null) { String timeColumn = timeExpression.getIdentifier().getName(); @@ -1661,7 +1670,7 @@ private static void fixColumnName(String rawTableName, Expression expression, Ma @VisibleForTesting static String getActualColumnName(String rawTableName, String columnName, @Nullable Map columnNameMap, boolean ignoreCase) { - if ("*".equals(columnName)) { + if ("*" .equals(columnName)) { return columnName; } String columnNameToCheck = trimTableName(rawTableName, columnName, ignoreCase); @@ -1872,10 +1881,8 @@ private static void attachTimeBoundary(PinotQuery pinotQuery, TimeBoundaryInfo t * TODO: Directly take PinotQuery */ protected abstract BrokerResponseNative processBrokerRequest(long requestId, BrokerRequest originalBrokerRequest, - BrokerRequest serverBrokerRequest, @Nullable BrokerRequest offlineBrokerRequest, - @Nullable Map, List>> offlineRoutingTable, - @Nullable BrokerRequest realtimeBrokerRequest, - @Nullable Map, List>> realtimeRoutingTable, long timeoutMs, + BrokerRequest serverBrokerRequest, + @Nullable Map> queryRoutingTable, long timeoutMs, ServerStats serverStats, RequestContext requestContext) throws Exception; @@ -1905,14 +1912,10 @@ private static class QueryServers { final String _query; final Set _servers = new HashSet<>(); - QueryServers(String query, @Nullable Map, List>> offlineRoutingTable, - @Nullable Map, List>> realtimeRoutingTable) { + QueryServers(String query, @Nullable Map> queryRoutingTable) { _query = query; - if (offlineRoutingTable != null) { - _servers.addAll(offlineRoutingTable.keySet()); - } - if (realtimeRoutingTable != null) { - _servers.addAll(realtimeRoutingTable.keySet()); + if (queryRoutingTable != null) { + _servers.addAll(queryRoutingTable.keySet()); } } } diff --git a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/GrpcBrokerRequestHandler.java b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/GrpcBrokerRequestHandler.java index 0484476a410..6b716412983 100644 --- a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/GrpcBrokerRequestHandler.java +++ b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/GrpcBrokerRequestHandler.java @@ -26,7 +26,6 @@ import java.util.concurrent.TimeUnit; import javax.annotation.Nullable; import javax.annotation.concurrent.ThreadSafe; -import org.apache.commons.lang3.tuple.Pair; import org.apache.pinot.broker.broker.AccessControlFactory; import org.apache.pinot.broker.queryquota.QueryQuotaManager; import org.apache.pinot.broker.routing.BrokerRoutingManager; @@ -39,8 +38,8 @@ import org.apache.pinot.common.utils.grpc.GrpcRequestBuilder; import org.apache.pinot.core.query.reduce.StreamingReduceService; import org.apache.pinot.core.transport.ServerInstance; +import org.apache.pinot.core.transport.ServerQueryRoutingContext; import org.apache.pinot.core.transport.ServerRoutingInstance; -import org.apache.pinot.spi.config.table.TableType; import org.apache.pinot.spi.env.PinotConfiguration; import org.apache.pinot.spi.trace.RequestContext; @@ -75,26 +74,15 @@ public void shutDown() { @Override protected BrokerResponseNative processBrokerRequest(long requestId, BrokerRequest originalBrokerRequest, - BrokerRequest serverBrokerRequest, @Nullable BrokerRequest offlineBrokerRequest, - @Nullable Map, List>> offlineRoutingTable, - @Nullable BrokerRequest realtimeBrokerRequest, - @Nullable Map, List>> realtimeRoutingTable, long timeoutMs, + BrokerRequest serverBrokerRequest, + @Nullable Map> queryRoutingTable, long timeoutMs, ServerStats serverStats, RequestContext requestContext) throws Exception { // TODO: Support failure detection // TODO: Add servers queried/responded stats - assert offlineBrokerRequest != null || realtimeBrokerRequest != null; + assert queryRoutingTable != null && !queryRoutingTable.isEmpty(); Map> responseMap = new HashMap<>(); - if (offlineBrokerRequest != null) { - assert offlineRoutingTable != null; - sendRequest(requestId, TableType.OFFLINE, offlineBrokerRequest, offlineRoutingTable, responseMap, - requestContext.isSampledRequest()); - } - if (realtimeBrokerRequest != null) { - assert realtimeRoutingTable != null; - sendRequest(requestId, TableType.REALTIME, realtimeBrokerRequest, realtimeRoutingTable, responseMap, - requestContext.isSampledRequest()); - } + sendRequest(requestId, queryRoutingTable, responseMap, requestContext.isSampledRequest()); long reduceStartTimeNs = System.nanoTime(); BrokerResponseNative brokerResponse = _streamingReduceService.reduceOnStreamResponse(originalBrokerRequest, responseMap, timeoutMs, _brokerMetrics); @@ -105,21 +93,21 @@ protected BrokerResponseNative processBrokerRequest(long requestId, BrokerReques /** * Query pinot server for data table. */ - private void sendRequest(long requestId, TableType tableType, BrokerRequest brokerRequest, - Map, List>> routingTable, + private void sendRequest(long requestId, Map> queryRoutingTable, Map> responseMap, boolean trace) { - for (Map.Entry, List>> routingEntry : routingTable.entrySet()) { + for (Map.Entry> routingEntry : queryRoutingTable.entrySet()) { ServerInstance serverInstance = routingEntry.getKey(); - // TODO: support optional segments for GrpcQueryServer. - List segments = routingEntry.getValue().getLeft(); - String serverHost = serverInstance.getHostname(); - int port = serverInstance.getGrpcPort(); - // TODO: enable throttling on per host bases. - Iterator streamingResponse = _streamingQueryClient.submit(serverHost, port, - new GrpcRequestBuilder().setRequestId(requestId).setBrokerId(_brokerId).setEnableTrace(trace) - .setEnableStreaming(true).setBrokerRequest(brokerRequest).setSegments(segments).build()); - responseMap.put(serverInstance.toServerRoutingInstance(tableType, ServerInstance.RoutingType.GRPC), - streamingResponse); + for (ServerQueryRoutingContext serverQueryRoutingContext : routingEntry.getValue()) { + // TODO: support optional segments for GrpcQueryServer. + String serverHost = serverInstance.getHostname(); + int port = serverInstance.getGrpcPort(); + // TODO: enable throttling on per host bases. + Iterator streamingResponse = _streamingQueryClient.submit(serverHost, port, + new GrpcRequestBuilder().setRequestId(requestId).setBrokerId(_brokerId).setEnableTrace(trace) + .setEnableStreaming(true).setBrokerRequest(serverQueryRoutingContext.getBrokerRequest()) + .setSegments(serverQueryRoutingContext.getRequiredSegmentsToQuery()).build()); + responseMap.put(serverInstance.toServerRoutingInstance(ServerInstance.RoutingType.GRPC), streamingResponse); + } } } diff --git a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/SingleConnectionBrokerRequestHandler.java b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/SingleConnectionBrokerRequestHandler.java index 349affe6c83..74b69e0ab81 100644 --- a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/SingleConnectionBrokerRequestHandler.java +++ b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/SingleConnectionBrokerRequestHandler.java @@ -20,12 +20,12 @@ import com.google.common.collect.Maps; import java.util.ArrayList; +import java.util.Collection; import java.util.List; import java.util.Map; import java.util.concurrent.TimeUnit; import javax.annotation.Nullable; import javax.annotation.concurrent.ThreadSafe; -import org.apache.commons.lang3.tuple.Pair; import org.apache.pinot.broker.broker.AccessControlFactory; import org.apache.pinot.broker.failuredetector.FailureDetector; import org.apache.pinot.broker.failuredetector.FailureDetectorFactory; @@ -39,6 +39,7 @@ import org.apache.pinot.common.metrics.BrokerMeter; import org.apache.pinot.common.metrics.BrokerQueryPhase; import org.apache.pinot.common.request.BrokerRequest; +import org.apache.pinot.core.transport.ServerQueryRoutingContext; import org.apache.pinot.common.response.broker.BrokerResponseNative; import org.apache.pinot.common.response.broker.QueryProcessingException; import org.apache.pinot.core.query.reduce.BrokerReduceService; @@ -97,13 +98,11 @@ public void shutDown() { @Override protected BrokerResponseNative processBrokerRequest(long requestId, BrokerRequest originalBrokerRequest, - BrokerRequest serverBrokerRequest, @Nullable BrokerRequest offlineBrokerRequest, - @Nullable Map, List>> offlineRoutingTable, - @Nullable BrokerRequest realtimeBrokerRequest, - @Nullable Map, List>> realtimeRoutingTable, long timeoutMs, + BrokerRequest serverBrokerRequest, + @Nullable Map> queryRoutingTable, long timeoutMs, ServerStats serverStats, RequestContext requestContext) throws Exception { - assert offlineBrokerRequest != null || realtimeBrokerRequest != null; + assert queryRoutingTable != null && !queryRoutingTable.isEmpty(); if (requestContext.isSampledRequest()) { serverBrokerRequest.getPinotQuery().putToQueryOptions(CommonConstants.Broker.Request.TRACE, "true"); } @@ -111,10 +110,9 @@ protected BrokerResponseNative processBrokerRequest(long requestId, BrokerReques String rawTableName = TableNameBuilder.extractRawTableName(serverBrokerRequest.getQuerySource().getTableName()); long scatterGatherStartTimeNs = System.nanoTime(); AsyncQueryResponse asyncQueryResponse = - _queryRouter.submitQuery(requestId, rawTableName, offlineBrokerRequest, offlineRoutingTable, - realtimeBrokerRequest, realtimeRoutingTable, timeoutMs); + _queryRouter.submitQuery(requestId, rawTableName, queryRoutingTable, timeoutMs); _failureDetector.notifyQuerySubmitted(asyncQueryResponse); - Map finalResponses = asyncQueryResponse.getFinalResponses(); + Map> finalResponses = asyncQueryResponse.getFinalResponses(); if (asyncQueryResponse.getStatus() == QueryResponse.Status.TIMED_OUT) { _brokerMetrics.addMeteredTableValue(rawTableName, BrokerMeter.BROKER_RESPONSES_WITH_TIMEOUTS, 1); } @@ -126,16 +124,18 @@ protected BrokerResponseNative processBrokerRequest(long requestId, BrokerReques int numServersQueried = finalResponses.size(); long totalResponseSize = 0; - Map dataTableMap = Maps.newHashMapWithExpectedSize(numServersQueried); + Map> dataTableMap = Maps.newHashMapWithExpectedSize(numServersQueried); List serversNotResponded = new ArrayList<>(); - for (Map.Entry entry : finalResponses.entrySet()) { - ServerResponse serverResponse = entry.getValue(); - DataTable dataTable = serverResponse.getDataTable(); - if (dataTable != null) { - dataTableMap.put(entry.getKey(), dataTable); - totalResponseSize += serverResponse.getResponseSize(); - } else { - serversNotResponded.add(entry.getKey()); + for (Map.Entry> serverResponses : finalResponses.entrySet()) { + for (Map.Entry responsePair : serverResponses.getValue().entrySet()) { + ServerResponse response = responsePair.getValue(); + DataTable dataTable = response.getDataTable(); + if (dataTable != null) { + dataTableMap.computeIfAbsent(serverResponses.getKey(), k -> new ArrayList<>()).add(dataTable); + totalResponseSize += response.getResponseSize(); + } else { + serversNotResponded.add(serverResponses.getKey()); + } } } int numServersResponded = dataTableMap.size(); diff --git a/pinot-common/src/main/java/org/apache/pinot/common/datatable/DataTable.java b/pinot-common/src/main/java/org/apache/pinot/common/datatable/DataTable.java index 890e2963494..caa1172457b 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/datatable/DataTable.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/datatable/DataTable.java @@ -136,11 +136,12 @@ enum MetadataKey { OPERATOR_ID(31, "operatorId", MetadataValueType.STRING), OPERATOR_EXEC_START_TIME_MS(32, "operatorExecStartTimeMs", MetadataValueType.LONG), OPERATOR_EXEC_END_TIME_MS(33, "operatorExecEndTimeMs", MetadataValueType.LONG), - MAX_ROWS_IN_JOIN_REACHED(34, "maxRowsInJoinReached", MetadataValueType.STRING); + MAX_ROWS_IN_JOIN_REACHED(34, "maxRowsInJoinReached", MetadataValueType.STRING), + QUERY_HASH(35, "queryHash", MetadataValueType.INT); // We keep this constant to track the max id added so far for backward compatibility. // Increase it when adding new keys, but NEVER DECREASE IT!!! - private static final int MAX_ID = 34; + private static final int MAX_ID = 35; private static final MetadataKey[] ID_TO_ENUM_KEY_MAP = new MetadataKey[MAX_ID + 1]; private static final Map NAME_TO_ENUM_KEY_MAP = new HashMap<>(); diff --git a/pinot-common/src/main/java/org/apache/pinot/common/datatable/DataTableImplV4.java b/pinot-common/src/main/java/org/apache/pinot/common/datatable/DataTableImplV4.java index a39a7d7e9f3..b426f34b32c 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/datatable/DataTableImplV4.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/datatable/DataTableImplV4.java @@ -93,6 +93,7 @@ public class DataTableImplV4 implements DataTable { protected ByteBuffer _fixedSizeData; protected byte[] _variableSizeDataBytes; protected ByteBuffer _variableSizeData; + // TODO(egalpin): add query hash to metadata, alongside requestId protected Map _metadata; protected int[] _columnOffsets; protected int _rowSizeInBytes; diff --git a/pinot-common/src/thrift/request.thrift b/pinot-common/src/thrift/request.thrift index 225836da549..141cb6d1e4c 100644 --- a/pinot-common/src/thrift/request.thrift +++ b/pinot-common/src/thrift/request.thrift @@ -52,4 +52,5 @@ struct InstanceRequest { 4: optional bool enableTrace; 5: optional string brokerId; 6: optional list optionalSegments; +// 7: required i64 queryHash; } diff --git a/pinot-connectors/pinot-spark-common/src/main/scala/org/apache/pinot/connector/spark/common/reader/PinotServerDataFetcher.scala b/pinot-connectors/pinot-spark-common/src/main/scala/org/apache/pinot/connector/spark/common/reader/PinotServerDataFetcher.scala index 1a0443203be..115b8ec587f 100644 --- a/pinot-connectors/pinot-spark-common/src/main/scala/org/apache/pinot/connector/spark/common/reader/PinotServerDataFetcher.scala +++ b/pinot-connectors/pinot-spark-common/src/main/scala/org/apache/pinot/connector/spark/common/reader/PinotServerDataFetcher.scala @@ -26,7 +26,7 @@ import org.apache.pinot.common.request.BrokerRequest import org.apache.pinot.connector.spark.common.partition.PinotSplit import org.apache.pinot.connector.spark.common.{Logging, PinotDataSourceReadOptions, PinotException} import org.apache.pinot.core.transport.server.routing.stats.ServerRoutingStatsManager -import org.apache.pinot.core.transport.{AsyncQueryResponse, QueryRouter, ServerInstance} +import org.apache.pinot.core.transport.{AsyncQueryResponse, QueryRouter, ServerInstance, ServerQueryRoutingContext} import org.apache.pinot.spi.config.table.TableType import org.apache.pinot.spi.env.PinotConfiguration import org.apache.pinot.spi.metrics.PinotMetricUtils @@ -106,18 +106,12 @@ private[reader] class PinotServerDataFetcher( } private def submitRequestToPinotServer( - offlineBrokerRequest: BrokerRequest, - offlineRoutingTable: JMap[ServerInstance, Pair[JList[String], JList[String]]], - realtimeBrokerRequest: BrokerRequest, - realtimeRoutingTable: JMap[ServerInstance, Pair[JList[String], JList[String]]]): AsyncQueryResponse = { + queryRoutingTable JMap[ServerInstance, JList[ServerQueryRoutingContext]]): AsyncQueryResponse = { logInfo(s"Sending request to ${pinotSplit.serverAndSegments.toString}") queryRouter.submitQuery( partitionId, pinotSplit.query.rawTableName, - offlineBrokerRequest, - offlineRoutingTable, - realtimeBrokerRequest, - realtimeRoutingTable, + queryRoutingTable, dataSourceOptions.pinotServerTimeoutMs ) } diff --git a/pinot-core/src/main/java/org/apache/pinot/core/query/executor/ServerQueryExecutorV1Impl.java b/pinot-core/src/main/java/org/apache/pinot/core/query/executor/ServerQueryExecutorV1Impl.java index 8c2906db541..c011e5074bc 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/query/executor/ServerQueryExecutorV1Impl.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/query/executor/ServerQueryExecutorV1Impl.java @@ -133,20 +133,26 @@ public synchronized void shutDown() { @Override public InstanceResponseBlock execute(ServerQueryRequest queryRequest, ExecutorService executorService, @Nullable ResultsBlockStreamer streamer) { + InstanceResponseBlock responseBlock; + if (!queryRequest.isEnableTrace()) { - return executeInternal(queryRequest, executorService, streamer); - } - try { - 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, streamer); - } finally { - Tracing.getTracer().unregister(); + responseBlock = executeInternal(queryRequest, executorService, streamer); + } else { + try { + 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); + responseBlock = executeInternal(queryRequest, executorService, streamer); + } finally { + Tracing.getTracer().unregister(); + } } + + responseBlock.addMetadata(MetadataKey.TABLE.getName(), queryRequest.getTableNameWithType()); + return responseBlock; } private InstanceResponseBlock executeInternal(ServerQueryRequest queryRequest, ExecutorService executorService, diff --git a/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/AggregationDataTableReducer.java b/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/AggregationDataTableReducer.java index 1c39b6971bf..7e5dc22c977 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/AggregationDataTableReducer.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/AggregationDataTableReducer.java @@ -57,7 +57,7 @@ public AggregationDataTableReducer(QueryContext queryContext) { */ @Override public void reduceAndSetResults(String tableName, DataSchema dataSchema, - Map dataTableMap, BrokerResponseNative brokerResponseNative, + Map> dataTableMap, BrokerResponseNative brokerResponseNative, DataTableReducerContext reducerContext, BrokerMetrics brokerMetrics) { dataSchema = ReducerDataSchemaUtils.canonicalizeDataSchemaForAggregation(_queryContext, dataSchema); @@ -68,7 +68,9 @@ public void reduceAndSetResults(String tableName, DataSchema dataSchema, return; } - Collection dataTables = dataTableMap.values(); + // flatten the data tables from all servers responding to the query into a single collection of DataTables + Collection dataTables = getFlatDataTables(dataTableMap); + if (_queryContext.isServerReturnFinalResult()) { if (dataTables.size() == 1) { processSingleFinalResult(dataSchema, dataTables.iterator().next(), brokerResponseNative); diff --git a/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/BrokerReduceService.java b/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/BrokerReduceService.java index 03d801941e3..ee68f1bdad5 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/BrokerReduceService.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/BrokerReduceService.java @@ -20,6 +20,7 @@ import java.util.ArrayList; import java.util.Arrays; +import java.util.Collection; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -59,7 +60,8 @@ public BrokerReduceService(PinotConfiguration config) { } public BrokerResponseNative reduceOnDataTable(BrokerRequest brokerRequest, BrokerRequest serverBrokerRequest, - Map dataTableMap, long reduceTimeOutMs, BrokerMetrics brokerMetrics) { + Map> dataTableMap, long reduceTimeOutMs, + BrokerMetrics brokerMetrics) { if (dataTableMap.isEmpty()) { // Empty response. return BrokerResponseNative.empty(); @@ -78,35 +80,38 @@ public BrokerResponseNative reduceOnDataTable(BrokerRequest brokerRequest, Broke List serversWithConflictingDataSchema = new ArrayList<>(); // Process server response metadata. - Iterator> iterator = dataTableMap.entrySet().iterator(); - while (iterator.hasNext()) { - Map.Entry entry = iterator.next(); - DataTable dataTable = entry.getValue(); - - // aggregate metrics - aggregator.aggregate(entry.getKey(), dataTable); - - // After processing the metadata, remove data tables without data rows inside. - DataSchema dataSchema = dataTable.getDataSchema(); - if (dataSchema == null) { - iterator.remove(); - } else { - // Try to cache a data table with data rows inside, or cache one with data schema inside. - if (dataTable.getNumberOfRows() == 0) { - if (dataSchemaFromEmptyDataTable == null) { - dataSchemaFromEmptyDataTable = dataSchema; - } - iterator.remove(); + for (Map.Entry> serverResponses : dataTableMap.entrySet()) { + Iterator tableIter = serverResponses.getValue().iterator(); + while (tableIter.hasNext()) { + DataTable dataTable = tableIter.next(); + ServerRoutingInstance serverRoutingInstance = serverResponses.getKey(); + + // aggregate metrics + aggregator.aggregate(serverRoutingInstance, dataTable); + + // After processing the metadata, remove data tables without data rows inside. + DataSchema dataSchema = dataTable.getDataSchema(); + if (dataSchema == null) { + tableIter.remove(); } else { - if (dataSchemaFromNonEmptyDataTable == null) { - dataSchemaFromNonEmptyDataTable = dataSchema; + // Try to cache a data table with data rows inside, or cache one with data schema inside. + if (dataTable.getNumberOfRows() == 0) { + if (dataSchemaFromEmptyDataTable == null) { + dataSchemaFromEmptyDataTable = dataSchema; + } + tableIter.remove(); } else { - // Remove data tables with conflicting data schema. - // NOTE: Only compare the column data types, since the column names (string representation of expression) - // can change across different versions. - if (!Arrays.equals(dataSchema.getColumnDataTypes(), dataSchemaFromNonEmptyDataTable.getColumnDataTypes())) { - serversWithConflictingDataSchema.add(entry.getKey()); - iterator.remove(); + if (dataSchemaFromNonEmptyDataTable == null) { + dataSchemaFromNonEmptyDataTable = dataSchema; + } else { + // Remove data tables with conflicting data schema. + // NOTE: Only compare the column data types, since the column names (string representation of expression) + // can change across different versions. + if (!Arrays.equals(dataSchema.getColumnDataTypes(), + dataSchemaFromNonEmptyDataTable.getColumnDataTypes())) { + serversWithConflictingDataSchema.add(serverRoutingInstance); + tableIter.remove(); + } } } } diff --git a/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/DataTableReducer.java b/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/DataTableReducer.java index 496df03da85..d23ea9e964e 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/DataTableReducer.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/DataTableReducer.java @@ -18,6 +18,9 @@ */ package org.apache.pinot.core.query.reduce; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; import java.util.Map; import org.apache.pinot.common.datatable.DataTable; import org.apache.pinot.common.metrics.BrokerMetrics; @@ -40,6 +43,14 @@ public interface DataTableReducer { * @param reducerContext DataTableReducer context * @param brokerMetrics broker metrics */ - void reduceAndSetResults(String tableName, DataSchema dataSchema, Map dataTableMap, - BrokerResponseNative brokerResponseNative, DataTableReducerContext reducerContext, BrokerMetrics brokerMetrics); + // TODO(egalpin): could dataTableMap be made into an Iterable instead? The keys appear unused in all impls + void reduceAndSetResults(String tableName, DataSchema dataSchema, + Map> dataTableMap, BrokerResponseNative brokerResponseNative, + DataTableReducerContext reducerContext, BrokerMetrics brokerMetrics); + + default Collection getFlatDataTables(Map> dataTableMap) { + List dataTables = new ArrayList<>(); + dataTableMap.values().forEach(dataTables::addAll); + return dataTables; + } } diff --git a/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/DistinctDataTableReducer.java b/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/DistinctDataTableReducer.java index 4553776963e..59b8a68b6e7 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/DistinctDataTableReducer.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/DistinctDataTableReducer.java @@ -19,6 +19,7 @@ package org.apache.pinot.core.query.reduce; import java.util.ArrayList; +import java.util.Collection; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -49,7 +50,7 @@ public DistinctDataTableReducer(QueryContext queryContext) { @Override public void reduceAndSetResults(String tableName, DataSchema dataSchema, - Map dataTableMap, BrokerResponseNative brokerResponseNative, + Map> dataTableMap, BrokerResponseNative brokerResponseNative, DataTableReducerContext reducerContext, BrokerMetrics brokerMetrics) { dataSchema = ReducerDataSchemaUtils.canonicalizeDataSchemaForDistinct(_queryContext, dataSchema); DistinctTable distinctTable = @@ -63,9 +64,10 @@ public void reduceAndSetResults(String tableName, DataSchema dataSchema, brokerResponseNative.setResultTable(reduceToResultTable(distinctTable)); } - private void addToOrderByDistinctTable(DataSchema dataSchema, Map dataTableMap, - DistinctTable distinctTable) { - for (DataTable dataTable : dataTableMap.values()) { + private void addToOrderByDistinctTable(DataSchema dataSchema, + Map> dataTableMap, DistinctTable distinctTable) { + + for (DataTable dataTable : getFlatDataTables(dataTableMap)) { Tracing.ThreadAccountantOps.sampleAndCheckInterruption(); int numColumns = dataSchema.size(); int numRows = dataTable.getNumberOfRows(); @@ -86,9 +88,9 @@ private void addToOrderByDistinctTable(DataSchema dataSchema, Map dataTableMap, - DistinctTable distinctTable) { - for (DataTable dataTable : dataTableMap.values()) { + private void addToNonOrderByDistinctTable(DataSchema dataSchema, + Map> dataTableMap, DistinctTable distinctTable) { + for (DataTable dataTable : getFlatDataTables(dataTableMap)) { Tracing.ThreadAccountantOps.sampleAndCheckInterruption(); int numColumns = dataSchema.size(); int numRows = dataTable.getNumberOfRows(); diff --git a/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/ExecutionStatsAggregator.java b/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/ExecutionStatsAggregator.java index 322af48d7bc..4f3d71ca31e 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/ExecutionStatsAggregator.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/ExecutionStatsAggregator.java @@ -32,6 +32,7 @@ import org.apache.pinot.common.response.broker.QueryProcessingException; import org.apache.pinot.core.transport.ServerRoutingInstance; import org.apache.pinot.spi.config.table.TableType; +import org.apache.pinot.spi.utils.builder.TableNameBuilder; public class ExecutionStatsAggregator { @@ -70,10 +71,11 @@ public ExecutionStatsAggregator(boolean enableTrace) { _enableTrace = enableTrace; } - public void aggregate(ServerRoutingInstance routingInstance, DataTable dataTable) { - TableType tableType = routingInstance.getTableType(); - String instanceName = routingInstance.getShortName(); + public void aggregate(ServerRoutingInstance serverRoutingInstance, DataTable dataTable) { + String instanceName = serverRoutingInstance.getShortName(); Map metadata = dataTable.getMetadata(); + TableType tableType = + TableNameBuilder.getTableTypeFromTableName(metadata.get(DataTable.MetadataKey.TABLE.getName())); // Reduce on trace info. if (_enableTrace && metadata.containsKey(DataTable.MetadataKey.TRACE_INFO.getName())) { diff --git a/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/ExplainPlanDataTableReducer.java b/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/ExplainPlanDataTableReducer.java index 97b2fbae824..d41615d2e64 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/ExplainPlanDataTableReducer.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/ExplainPlanDataTableReducer.java @@ -19,6 +19,7 @@ package org.apache.pinot.core.query.reduce; import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; import java.util.HashSet; import java.util.List; @@ -55,7 +56,7 @@ public class ExplainPlanDataTableReducer implements DataTableReducer { @Override public void reduceAndSetResults(String tableName, DataSchema dataSchema, - Map dataTableMap, BrokerResponseNative brokerResponseNative, + Map> dataTableMap, BrokerResponseNative brokerResponseNative, DataTableReducerContext reducerContext, BrokerMetrics brokerMetrics) { List reducedRows = new ArrayList<>(); @@ -103,14 +104,13 @@ public void reduceAndSetResults(String tableName, DataSchema dataSchema, * Extract the combine node to use as the global combine step if present. If no combine node is found, return null. * A combine node may not be found if all segments were pruned across all servers. */ - private Object[] extractCombineNode(Map dataTableMap) { + private Object[] extractCombineNode(Map> dataTableMap) { if (dataTableMap.isEmpty()) { return null; } Object[] combineRow = null; - for (Map.Entry entry : dataTableMap.entrySet()) { - DataTable dataTable = entry.getValue(); + for (DataTable dataTable : getFlatDataTables(dataTableMap)) { int numRows = dataTable.getNumberOfRows(); if (numRows > 0) { // First row should be the combine row data, unless all segments were pruned from the Server side @@ -129,12 +129,11 @@ private Object[] extractCombineNode(Map dataTa * Extract a list of all the unique explain plans across all servers */ private List extractUniqueExplainPlansAcrossServers( - Map dataTableMap, Object[] combinedRow) { + Map> dataTableMap, Object[] combinedRow) { List explainPlanRowsList = new ArrayList<>(); HashSet explainPlanHashCodeSet = new HashSet<>(); - for (Map.Entry entry : dataTableMap.entrySet()) { - DataTable dataTable = entry.getValue(); + for (DataTable dataTable : getFlatDataTables(dataTableMap)) { int numRows = dataTable.getNumberOfRows(); ExplainPlanRows explainPlanRows = null; diff --git a/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/GroupByDataTableReducer.java b/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/GroupByDataTableReducer.java index 09b4d6a1561..669f32cf264 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/GroupByDataTableReducer.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/GroupByDataTableReducer.java @@ -101,7 +101,7 @@ public GroupByDataTableReducer(QueryContext queryContext) { */ @Override public void reduceAndSetResults(String tableName, DataSchema dataSchema, - Map dataTableMap, BrokerResponseNative brokerResponse, + Map> dataTableMap, BrokerResponseNative brokerResponse, DataTableReducerContext reducerContext, BrokerMetrics brokerMetrics) { dataSchema = ReducerDataSchemaUtils.canonicalizeDataSchemaForGroupBy(_queryContext, dataSchema); @@ -114,7 +114,7 @@ public void reduceAndSetResults(String tableName, DataSchema dataSchema, return; } - Collection dataTables = dataTableMap.values(); + Collection dataTables = getFlatDataTables(dataTableMap); // NOTE: Use regular reduce when group keys are not partitioned even if there are only one data table because the // records are not sorted yet. if (_queryContext.isServerReturnFinalResult() && dataTables.size() == 1) { diff --git a/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/SelectionDataTableReducer.java b/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/SelectionDataTableReducer.java index a8226aee5e2..825c90a1aec 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/SelectionDataTableReducer.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/SelectionDataTableReducer.java @@ -18,6 +18,7 @@ */ package org.apache.pinot.core.query.reduce; +import java.util.Collection; import java.util.Collections; import java.util.List; import java.util.Map; @@ -48,7 +49,7 @@ public SelectionDataTableReducer(QueryContext queryContext) { */ @Override public void reduceAndSetResults(String tableName, DataSchema dataSchema, - Map dataTableMap, BrokerResponseNative brokerResponseNative, + Map> dataTableMap, BrokerResponseNative brokerResponseNative, DataTableReducerContext reducerContext, BrokerMetrics brokerMetrics) { Pair pair = SelectionOperatorUtils.getResultTableDataSchemaAndColumnIndices(_queryContext, dataSchema); @@ -59,7 +60,7 @@ public void reduceAndSetResults(String tableName, DataSchema dataSchema, } if (_queryContext.getOrderByExpressions() == null) { // Selection only - List reducedRows = SelectionOperatorUtils.reduceWithoutOrdering(dataTableMap.values(), limit, + List reducedRows = SelectionOperatorUtils.reduceWithoutOrdering(getFlatDataTables(dataTableMap), limit, _queryContext.isNullHandlingEnabled()); brokerResponseNative.setResultTable( SelectionOperatorUtils.renderResultTableWithoutOrdering(reducedRows, pair.getLeft(), pair.getRight())); @@ -67,7 +68,7 @@ public void reduceAndSetResults(String tableName, DataSchema dataSchema, // Selection order-by SelectionOperatorService selectionService = new SelectionOperatorService(_queryContext, pair.getLeft(), pair.getRight()); - selectionService.reduceWithOrdering(dataTableMap.values()); + selectionService.reduceWithOrdering(getFlatDataTables(dataTableMap)); brokerResponseNative.setResultTable(selectionService.renderResultTableWithOrdering()); } } diff --git a/pinot-core/src/main/java/org/apache/pinot/core/query/request/ServerQueryRequest.java b/pinot-core/src/main/java/org/apache/pinot/core/query/request/ServerQueryRequest.java index 87586fe1541..1610db91376 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/query/request/ServerQueryRequest.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/query/request/ServerQueryRequest.java @@ -44,6 +44,7 @@ */ public class ServerQueryRequest { private final long _requestId; + private final int _queryHash; private final String _brokerId; private final boolean _enableTrace; private final boolean _enableStreaming; @@ -71,6 +72,9 @@ public ServerQueryRequest(InstanceRequest instanceRequest, ServerMetrics serverM _segmentsToQuery = instanceRequest.getSearchSegments(); _optionalSegments = instanceRequest.getOptionalSegments(); _queryContext = getQueryContext(instanceRequest.getQuery().getPinotQuery()); + // TODO(egalpin): instanceRequest.getQuery() or instanceRequest.getQuery().getPinotQuery() ? + // needs to match whats in AsyncQueryResponse + _queryHash = instanceRequest.getQuery().getPinotQuery().hashCode(); _queryId = QueryIdUtils.getQueryId(_brokerId, _requestId, TableNameBuilder.getTableTypeFromTableName(_queryContext.getTableName())); _timerContext = new TimerContext(_queryContext.getTableName(), serverMetrics, queryArrivalTimeMs); @@ -102,6 +106,7 @@ public ServerQueryRequest(Server.ServerRequest serverRequest, ServerMetrics serv throw new UnsupportedOperationException("Unsupported payloadType: " + payloadType); } _queryContext = getQueryContext(brokerRequest.getPinotQuery()); + _queryHash = brokerRequest.getPinotQuery().hashCode(); _queryId = QueryIdUtils.getQueryId(_brokerId, _requestId, TableNameBuilder.getTableTypeFromTableName(_queryContext.getTableName())); _timerContext = new TimerContext(_queryContext.getTableName(), serverMetrics, queryArrivalTimeMs); @@ -115,6 +120,10 @@ public long getRequestId() { return _requestId; } + public int getQueryHash() { + return _queryHash; + } + public String getBrokerId() { return _brokerId; } diff --git a/pinot-core/src/main/java/org/apache/pinot/core/query/scheduler/QueryScheduler.java b/pinot-core/src/main/java/org/apache/pinot/core/query/scheduler/QueryScheduler.java index f4c2a241e76..97a250ea110 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/query/scheduler/QueryScheduler.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/query/scheduler/QueryScheduler.java @@ -131,6 +131,7 @@ protected ListenableFutureTask createQueryFutureTask(ServerQueryRequest * @return serialized query response */ @Nullable + // egalpin: where dataTable is created and serialized protected byte[] processQueryAndSerialize(ServerQueryRequest queryRequest, ExecutorService executorService) { //Start instrumentation context. This must not be moved further below interspersed into the code. @@ -153,6 +154,7 @@ protected byte[] processQueryAndSerialize(ServerQueryRequest queryRequest, Execu long requestId = queryRequest.getRequestId(); Map responseMetadata = instanceResponse.getResponseMetadata(); responseMetadata.put(MetadataKey.REQUEST_ID.getName(), Long.toString(requestId)); + responseMetadata.put(MetadataKey.QUERY_HASH.getName(), Integer.toString(queryRequest.getQueryHash())); byte[] responseBytes = serializeResponse(queryRequest, instanceResponse); // Log the statistics diff --git a/pinot-core/src/main/java/org/apache/pinot/core/transport/AsyncQueryResponse.java b/pinot-core/src/main/java/org/apache/pinot/core/transport/AsyncQueryResponse.java index 2ec90ab3b90..000207752bd 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/transport/AsyncQueryResponse.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/transport/AsyncQueryResponse.java @@ -18,8 +18,9 @@ */ package org.apache.pinot.core.transport; +import java.util.HashMap; +import java.util.List; import java.util.Map; -import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; @@ -28,7 +29,7 @@ import javax.annotation.Nullable; import javax.annotation.concurrent.ThreadSafe; import org.apache.pinot.common.datatable.DataTable; -import org.apache.pinot.common.utils.HashUtil; +import org.apache.pinot.common.request.InstanceRequest; import org.apache.pinot.core.transport.server.routing.stats.ServerRoutingStatsManager; @@ -41,7 +42,7 @@ public class AsyncQueryResponse implements QueryResponse { private final long _requestId; private final AtomicReference _status = new AtomicReference<>(Status.IN_PROGRESS); private final AtomicInteger _numServersResponded = new AtomicInteger(); - private final ConcurrentHashMap _responseMap; + private final Map> _responses; private final CountDownLatch _countDownLatch; private final long _maxEndTimeMs; private final long _timeoutMs; @@ -50,21 +51,31 @@ public class AsyncQueryResponse implements QueryResponse { private volatile ServerRoutingInstance _failedServer; private volatile Exception _exception; - public AsyncQueryResponse(QueryRouter queryRouter, long requestId, Set serversQueried, - long startTimeMs, long timeoutMs, ServerRoutingStatsManager serverRoutingStatsManager) { + public AsyncQueryResponse(QueryRouter queryRouter, long requestId, +// Set serverQueryRoutingContexts, + Map> requestMap, long startTimeMs, long timeoutMs, + ServerRoutingStatsManager serverRoutingStatsManager) { _queryRouter = queryRouter; _requestId = requestId; - int numServersQueried = serversQueried.size(); - _responseMap = new ConcurrentHashMap<>(HashUtil.getHashMapCapacity(numServersQueried)); + _responses = new ConcurrentHashMap<>(); _serverRoutingStatsManager = serverRoutingStatsManager; - for (ServerRoutingInstance serverRoutingInstance : serversQueried) { - // Record stats related to query submission just before sending the request. Otherwise, if the response is - // received immediately, there's a possibility of updating query response stats before updating query - // submission stats. - _serverRoutingStatsManager.recordStatsForQuerySubmission(requestId, serverRoutingInstance.getInstanceId()); - _responseMap.put(serverRoutingInstance, new ServerResponse(startTimeMs)); + int numQueriesIssued = 0; + for (Map.Entry> serverRequests : requestMap.entrySet()) { + for (InstanceRequest request : serverRequests.getValue()) { + // Record stats related to query submission just before sending the request. Otherwise, if the response is + // received immediately, there's a possibility of updating query response stats before updating query + // submission stats. + _serverRoutingStatsManager.recordStatsForQuerySubmission(requestId, serverRequests.getKey().getInstanceId()); + + _responses.computeIfAbsent(serverRequests.getKey(), k -> new HashMap<>()) + // we use query hash so that the same hash ID can be passed back from servers more easily than trying to + // instantiate a valid InstanceRequest obj and send its hash + .put(request.getQuery().getPinotQuery().hashCode(), new ServerResponse(startTimeMs)); + numQueriesIssued++; + } } - _countDownLatch = new CountDownLatch(numServersQueried); + + _countDownLatch = new CountDownLatch(numQueriesIssued); _timeoutMs = timeoutMs; _maxEndTimeMs = startTimeMs + timeoutMs; } @@ -80,30 +91,32 @@ public int getNumServersResponded() { } @Override - public Map getCurrentResponses() { - return _responseMap; + public Map> getCurrentResponses() { + return _responses; } @Override - public Map getFinalResponses() + public Map> getFinalResponses() throws InterruptedException { try { boolean finish = _countDownLatch.await(_maxEndTimeMs - System.currentTimeMillis(), TimeUnit.MILLISECONDS); _status.compareAndSet(Status.IN_PROGRESS, finish ? Status.COMPLETED : Status.TIMED_OUT); - return _responseMap; + return _responses; } finally { // Update ServerRoutingStats for query completion. This is done here to ensure that the stats are updated for // servers even if the query times out or if servers have not responded. - for (Map.Entry entry : _responseMap.entrySet()) { - ServerResponse response = entry.getValue(); - - // ServerResponse returns -1 if responseDelayMs is not set. This indicates that a response was not received - // from the server. Hence we set the latency to the timeout value. - long latency = - (response != null && response.getResponseDelayMs() >= 0) ? response.getResponseDelayMs() : _timeoutMs; - _serverRoutingStatsManager.recordStatsUponResponseArrival(_requestId, entry.getKey().getInstanceId(), latency); + for (Map.Entry> serverResponses : _responses.entrySet()) { + for (Map.Entry responsePair : serverResponses.getValue().entrySet()) { + ServerResponse response = responsePair.getValue(); + + // ServerResponse returns -1 if responseDelayMs is not set. This indicates that a response was not received + // from the server. Hence we set the latency to the timeout value. + long latency = + (response != null && response.getResponseDelayMs() >= 0) ? response.getResponseDelayMs() : _timeoutMs; + _serverRoutingStatsManager.recordStatsUponResponseArrival(_requestId, + serverResponses.getKey().getInstanceId(), latency); + } } - _queryRouter.markQueryDone(_requestId); } } @@ -112,15 +125,21 @@ public Map getFinalResponses() public String getServerStats() { StringBuilder stringBuilder = new StringBuilder( "(Server=SubmitDelayMs,ResponseDelayMs,ResponseSize,DeserializationTimeMs,RequestSentDelayMs)"); - for (Map.Entry entry : _responseMap.entrySet()) { - stringBuilder.append(';').append(entry.getKey().getShortName()).append('=').append(entry.getValue().toString()); + for (Map.Entry> serverResponses : _responses.entrySet()) { + for (Map.Entry responsePair : serverResponses.getValue().entrySet()) { + ServerRoutingInstance serverRoutingInstance = serverResponses.getKey(); + stringBuilder.append(';').append(serverRoutingInstance.getShortName()).append('=') + .append(responsePair.getValue().toString()); + } } return stringBuilder.toString(); } @Override public long getServerResponseDelayMs(ServerRoutingInstance serverRoutingInstance) { - return _responseMap.get(serverRoutingInstance).getResponseDelayMs(); + // TODO(egalpin): How to get query hash here? + return -1L; +// return _responseMap.get(serverRoutingInstance).getResponseDelayMs(); } @Nullable @@ -144,21 +163,23 @@ public long getTimeoutMs() { return _timeoutMs; } - void markRequestSubmitted(ServerRoutingInstance serverRoutingInstance) { - _responseMap.get(serverRoutingInstance).markRequestSubmitted(); + void markRequestSubmitted(ServerRoutingInstance serverRoutingInstance, InstanceRequest instanceRequest) { + _responses.get(serverRoutingInstance).get(instanceRequest.getQuery().getPinotQuery().hashCode()) + .markRequestSubmitted(); } - void markRequestSent(ServerRoutingInstance serverRoutingInstance, int requestSentLatencyMs) { - _responseMap.get(serverRoutingInstance).markRequestSent(requestSentLatencyMs); + void markRequestSent(ServerRoutingInstance serverRoutingInstance, InstanceRequest instanceRequest, + int requestSentLatencyMs) { + _responses.get(serverRoutingInstance).get(instanceRequest.getQuery().hashCode()) + .markRequestSent(requestSentLatencyMs); } - void receiveDataTable(ServerRoutingInstance serverRoutingInstance, DataTable dataTable, int responseSize, - int deserializationTimeMs) { - ServerResponse response = _responseMap.get(serverRoutingInstance); + void receiveDataTable(ServerRoutingInstance serverRoutingInstance, int queryHash, DataTable dataTable, + int responseSize, int deserializationTimeMs) { + ServerResponse response = _responses.get(serverRoutingInstance).get(queryHash); response.receiveDataTable(dataTable, responseSize, deserializationTimeMs); - - _numServersResponded.getAndIncrement(); _countDownLatch.countDown(); + _numServersResponded.getAndIncrement(); } void markQueryFailed(ServerRoutingInstance serverRoutingInstance, Exception exception) { @@ -176,9 +197,13 @@ void markQueryFailed(ServerRoutingInstance serverRoutingInstance, Exception exce * server hasn't responded yet. */ void markServerDown(ServerRoutingInstance serverRoutingInstance, Exception exception) { - ServerResponse serverResponse = _responseMap.get(serverRoutingInstance); - if (serverResponse != null && serverResponse.getDataTable() == null) { - markQueryFailed(serverRoutingInstance, exception); + // TODO(egalpin): how to make servers down under the assumption that multiple queries + // to the same server are valid? + Map serverResponses = _responses.get(serverRoutingInstance); + for (ServerResponse serverResponse : serverResponses.values()) { + if (serverResponse != null && serverResponse.getDataTable() == null) { + markQueryFailed(serverRoutingInstance, exception); + } } } diff --git a/pinot-core/src/main/java/org/apache/pinot/core/transport/DirectOOMHandler.java b/pinot-core/src/main/java/org/apache/pinot/core/transport/DirectOOMHandler.java index 0ba00327f63..d120a1f68fd 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/transport/DirectOOMHandler.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/transport/DirectOOMHandler.java @@ -73,7 +73,8 @@ public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws E if (DIRECT_OOM_SHUTTING_DOWN.compareAndSet(false, true)) { try { LOGGER.error("Closing ALL channels to servers, as we are running out of direct memory " - + "while receiving response from {}", _serverRoutingInstance, cause); + + "while receiving response from {}", + _serverRoutingInstance, cause); // close all channels to servers _serverToChannelMap.keySet().forEach(serverRoutingInstance -> { ServerChannels.ServerChannel removed = _serverToChannelMap.remove(serverRoutingInstance); diff --git a/pinot-core/src/main/java/org/apache/pinot/core/transport/InstanceRequestHandler.java b/pinot-core/src/main/java/org/apache/pinot/core/transport/InstanceRequestHandler.java index 4940823b741..b25c1aabda0 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/transport/InstanceRequestHandler.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/transport/InstanceRequestHandler.java @@ -65,6 +65,9 @@ * The {@code InstanceRequestHandler} is the Netty inbound handler on Pinot Server side to handle the serialized * instance requests sent from Pinot Broker. */ + +// TODO(egalpin): server-side handler for responding to requests sent from broker + @ChannelHandler.Sharable public class InstanceRequestHandler extends SimpleChannelInboundHandler { private static final Logger LOGGER = LoggerFactory.getLogger(InstanceRequestHandler.class); @@ -183,6 +186,8 @@ void submitQuery(ServerQueryRequest queryRequest, ChannelHandlerContext ctx, Str private FutureCallback createCallback(ChannelHandlerContext ctx, String tableNameWithType, long queryArrivalTimeMs, InstanceRequest instanceRequest, ServerQueryRequest queryRequest) { + // egalpin: whatever invokes this callback has constructed the data table, where we need to add queryHash to the + // metadata return new FutureCallback() { @Override public void onSuccess(@Nullable byte[] responseBytes) { @@ -323,8 +328,8 @@ private void sendResponse(ChannelHandlerContext ctx, String tableNameWithType, l + "request: {}", queryProcessingTimeMs, sendResponseLatencyMs, totalQueryTimeMs); } if (serializedDataTable.length > LARGE_RESPONSE_SIZE_THRESHOLD_BYTES) { - LOGGER.warn("Large query: response size in bytes: {}, table name {}", - serializedDataTable.length, tableNameWithType); + LOGGER.warn("Large query: response size in bytes: {}, table name {}", serializedDataTable.length, + tableNameWithType); ServerMetrics.get().addMeteredTableValue(tableNameWithType, ServerMeter.LARGE_QUERY_RESPONSES_SENT, 1); } }); diff --git a/pinot-core/src/main/java/org/apache/pinot/core/transport/QueryResponse.java b/pinot-core/src/main/java/org/apache/pinot/core/transport/QueryResponse.java index 23f5946cd37..14cd932ea48 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/transport/QueryResponse.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/transport/QueryResponse.java @@ -49,12 +49,12 @@ enum Status { /** * Returns the current server responses without blocking. */ - Map getCurrentResponses(); + Map> getCurrentResponses(); /** * Waits until the query is done (COMPLETED, FAILED or TIMED_OUT) and returns the final server responses. */ - Map getFinalResponses() + Map> getFinalResponses() throws InterruptedException; /** diff --git a/pinot-core/src/main/java/org/apache/pinot/core/transport/QueryRouter.java b/pinot-core/src/main/java/org/apache/pinot/core/transport/QueryRouter.java index 2b756118652..1d221dc7323 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/transport/QueryRouter.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/transport/QueryRouter.java @@ -18,6 +18,7 @@ */ package org.apache.pinot.core.transport; +import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -37,7 +38,6 @@ import org.apache.pinot.common.request.InstanceRequest; import org.apache.pinot.common.utils.config.QueryOptionsUtils; import org.apache.pinot.core.transport.server.routing.stats.ServerRoutingStatsManager; -import org.apache.pinot.spi.config.table.TableType; import org.apache.pinot.spi.utils.CommonConstants; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -88,64 +88,63 @@ public QueryRouter(String brokerId, BrokerMetrics brokerMetrics, @Nullable Netty } public AsyncQueryResponse submitQuery(long requestId, String rawTableName, - @Nullable BrokerRequest offlineBrokerRequest, - @Nullable Map, List>> offlineRoutingTable, - @Nullable BrokerRequest realtimeBrokerRequest, - @Nullable Map, List>> realtimeRoutingTable, long timeoutMs) { - assert offlineBrokerRequest != null || realtimeBrokerRequest != null; + @Nullable Map> queryRoutingTable, long timeoutMs) { + assert queryRoutingTable != null && !queryRoutingTable.isEmpty(); // can prefer but not require TLS until all servers guaranteed to be on TLS boolean preferTls = _serverChannelsTls != null; // skip unavailable servers if the query option is set - boolean skipUnavailableServers = isSkipUnavailableServers(offlineBrokerRequest, realtimeBrokerRequest); + boolean skipUnavailableServers = false; // Build map from server to request based on the routing table - Map requestMap = new HashMap<>(); - if (offlineBrokerRequest != null) { - assert offlineRoutingTable != null; - for (Map.Entry, List>> entry : offlineRoutingTable.entrySet()) { - ServerRoutingInstance serverRoutingInstance = - entry.getKey().toServerRoutingInstance(TableType.OFFLINE, preferTls); - InstanceRequest instanceRequest = getInstanceRequest(requestId, offlineBrokerRequest, entry.getValue()); - requestMap.put(serverRoutingInstance, instanceRequest); - } - } - if (realtimeBrokerRequest != null) { - assert realtimeRoutingTable != null; - for (Map.Entry, List>> entry : realtimeRoutingTable.entrySet()) { - ServerRoutingInstance serverRoutingInstance = - entry.getKey().toServerRoutingInstance(TableType.REALTIME, preferTls); - InstanceRequest instanceRequest = getInstanceRequest(requestId, realtimeBrokerRequest, entry.getValue()); - requestMap.put(serverRoutingInstance, instanceRequest); + Map> requestMap = new HashMap<>(); + for (Map.Entry> entry : queryRoutingTable.entrySet()) { + for (ServerQueryRoutingContext serverQueryRoutingContext : entry.getValue()) { + ServerRoutingInstance serverRoutingInstance = entry.getKey().toServerRoutingInstance(preferTls); + InstanceRequest instanceRequest = getInstanceRequest(requestId, serverQueryRoutingContext.getBrokerRequest(), + serverQueryRoutingContext.getSegmentsToQuery()); + if (!skipUnavailableServers && QueryOptionsUtils.isSkipUnavailableServers( + serverQueryRoutingContext.getBrokerRequest().getPinotQuery().getQueryOptions())) { + // Any single query having this option set will set it for all queries. This option should not be possible + // to differ between server instance requests pertaining to the same request ID. + skipUnavailableServers = true; + } + + // TODO(egalpin): Revamp serverRoutingInstance hashCode to include broker request some how? + // TODO(egalpin): This is wrong: there can be many instance requests to the same server i.e. + // serverRoutingInstance + requestMap.computeIfAbsent(serverRoutingInstance, k -> new ArrayList<>()).add(instanceRequest); } } // Create the asynchronous query response with the request map AsyncQueryResponse asyncQueryResponse = - new AsyncQueryResponse(this, requestId, requestMap.keySet(), System.currentTimeMillis(), timeoutMs, + new AsyncQueryResponse(this, requestId, requestMap, System.currentTimeMillis(), timeoutMs, _serverRoutingStatsManager); _asyncQueryResponseMap.put(requestId, asyncQueryResponse); - for (Map.Entry entry : requestMap.entrySet()) { - ServerRoutingInstance serverRoutingInstance = entry.getKey(); - ServerChannels serverChannels = serverRoutingInstance.isTlsEnabled() ? _serverChannelsTls : _serverChannels; - try { - serverChannels.sendRequest(rawTableName, asyncQueryResponse, serverRoutingInstance, entry.getValue(), - timeoutMs); - asyncQueryResponse.markRequestSubmitted(serverRoutingInstance); - } catch (TimeoutException e) { - if (ServerChannels.CHANNEL_LOCK_TIMEOUT_MSG.equals(e.getMessage())) { - _brokerMetrics.addMeteredTableValue(rawTableName, BrokerMeter.REQUEST_CHANNEL_LOCK_TIMEOUT_EXCEPTIONS, 1); - } - markQueryFailed(requestId, serverRoutingInstance, asyncQueryResponse, e); - break; - } catch (Exception e) { - _brokerMetrics.addMeteredTableValue(rawTableName, BrokerMeter.REQUEST_SEND_EXCEPTIONS, 1); - if (skipUnavailableServers) { - asyncQueryResponse.skipServerResponse(); - } else { + for (Map.Entry> serverRequests : requestMap.entrySet()) { + for (InstanceRequest request : serverRequests.getValue()) { + + ServerRoutingInstance serverRoutingInstance = serverRequests.getKey(); + ServerChannels serverChannels = serverRoutingInstance.isTlsEnabled() ? _serverChannelsTls : _serverChannels; + try { + serverChannels.sendRequest(rawTableName, asyncQueryResponse, serverRoutingInstance, request, timeoutMs); + asyncQueryResponse.markRequestSubmitted(serverRoutingInstance, request); + } catch (TimeoutException e) { + if (ServerChannels.CHANNEL_LOCK_TIMEOUT_MSG.equals(e.getMessage())) { + _brokerMetrics.addMeteredTableValue(rawTableName, BrokerMeter.REQUEST_CHANNEL_LOCK_TIMEOUT_EXCEPTIONS, 1); + } markQueryFailed(requestId, serverRoutingInstance, asyncQueryResponse, e); break; + } catch (Exception e) { + _brokerMetrics.addMeteredTableValue(rawTableName, BrokerMeter.REQUEST_SEND_EXCEPTIONS, 1); + if (skipUnavailableServers) { + asyncQueryResponse.skipServerResponse(); + } else { + markQueryFailed(requestId, serverRoutingInstance, asyncQueryResponse, e); + break; + } } } } @@ -153,16 +152,6 @@ public AsyncQueryResponse submitQuery(long requestId, String rawTableName, return asyncQueryResponse; } - private boolean isSkipUnavailableServers(@Nullable BrokerRequest offlineBrokerRequest, - @Nullable BrokerRequest realtimeBrokerRequest) { - if (offlineBrokerRequest != null && QueryOptionsUtils.isSkipUnavailableServers( - offlineBrokerRequest.getPinotQuery().getQueryOptions())) { - return true; - } - return realtimeBrokerRequest != null && QueryOptionsUtils.isSkipUnavailableServers( - realtimeBrokerRequest.getPinotQuery().getQueryOptions()); - } - private void markQueryFailed(long requestId, ServerRoutingInstance serverRoutingInstance, AsyncQueryResponse asyncQueryResponse, Exception e) { LOGGER.error("Caught exception while sending request {} to server: {}, marking query failed", requestId, @@ -176,9 +165,9 @@ private void markQueryFailed(long requestId, ServerRoutingInstance serverRouting public boolean connect(ServerInstance serverInstance) { try { if (_serverChannelsTls != null) { - _serverChannelsTls.connect(serverInstance.toServerRoutingInstance(TableType.OFFLINE, true)); + _serverChannelsTls.connect(serverInstance.toServerRoutingInstance(true)); } else { - _serverChannels.connect(serverInstance.toServerRoutingInstance(TableType.OFFLINE, false)); + _serverChannels.connect(serverInstance.toServerRoutingInstance(false)); } return true; } catch (Exception e) { @@ -193,12 +182,15 @@ public void shutDown() { void receiveDataTable(ServerRoutingInstance serverRoutingInstance, DataTable dataTable, int responseSize, int deserializationTimeMs) { + // TODO(egalpin): read queryHash from dataTable metadata long requestId = Long.parseLong(dataTable.getMetadata().get(MetadataKey.REQUEST_ID.getName())); + int queryHash = Integer.parseInt(dataTable.getMetadata().get(MetadataKey.QUERY_HASH.getName())); AsyncQueryResponse asyncQueryResponse = _asyncQueryResponseMap.get(requestId); // Query future might be null if the query is already done (maybe due to failure) if (asyncQueryResponse != null) { - asyncQueryResponse.receiveDataTable(serverRoutingInstance, dataTable, responseSize, deserializationTimeMs); + asyncQueryResponse.receiveDataTable(serverRoutingInstance, queryHash, dataTable, responseSize, + deserializationTimeMs); } } diff --git a/pinot-core/src/main/java/org/apache/pinot/core/transport/ServerChannels.java b/pinot-core/src/main/java/org/apache/pinot/core/transport/ServerChannels.java index c9fe068ed44..03f19b2f2c9 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/transport/ServerChannels.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/transport/ServerChannels.java @@ -87,15 +87,11 @@ public ServerChannels(QueryRouter queryRouter, BrokerMetrics brokerMetrics, @Nul @Nullable TlsConfig tlsConfig) { boolean enableNativeTransports = nettyConfig != null && nettyConfig.isNativeTransportsEnabled(); OsCheck.OSType operatingSystemType = OsCheck.getOperatingSystemType(); - if (enableNativeTransports - && operatingSystemType == OsCheck.OSType.Linux - && Epoll.isAvailable()) { + if (enableNativeTransports && operatingSystemType == OsCheck.OSType.Linux && Epoll.isAvailable()) { _eventLoopGroup = new EpollEventLoopGroup(); _channelClass = EpollSocketChannel.class; LOGGER.info("Using Epoll event loop"); - } else if (enableNativeTransports - && operatingSystemType == OsCheck.OSType.MacOS - && KQueue.isAvailable()) { + } else if (enableNativeTransports && operatingSystemType == OsCheck.OSType.MacOS && KQueue.isAvailable()) { _eventLoopGroup = new KQueueEventLoopGroup(); _channelClass = KQueueSocketChannel.class; LOGGER.info("Using KQueue event loop"); @@ -103,11 +99,9 @@ public ServerChannels(QueryRouter queryRouter, BrokerMetrics brokerMetrics, @Nul _eventLoopGroup = new NioEventLoopGroup(); _channelClass = NioSocketChannel.class; StringBuilder log = new StringBuilder("Using NIO event loop"); - if (operatingSystemType == OsCheck.OSType.Linux - && enableNativeTransports) { + if (operatingSystemType == OsCheck.OSType.Linux && enableNativeTransports) { log.append(", as Epoll is not available: ").append(Epoll.unavailabilityCause()); - } else if (operatingSystemType == OsCheck.OSType.MacOS - && enableNativeTransports) { + } else if (operatingSystemType == OsCheck.OSType.MacOS && enableNativeTransports) { log.append(", as KQueue is not available: ").append(KQueue.unavailabilityCause()); } LOGGER.info(log.toString()); @@ -130,7 +124,9 @@ public void sendRequest(String rawTableName, AsyncQueryResponse asyncQueryRespon throws Exception { byte[] requestBytes = _threadLocalTSerializer.get().serialize(instanceRequest); _serverToChannelMap.computeIfAbsent(serverRoutingInstance, ServerChannel::new) - .sendRequest(rawTableName, asyncQueryResponse, serverRoutingInstance, requestBytes, timeoutMs); + // TODO(egalpin): what if serverRoutingInstance was an instance of ServerQueryRoutingContext instead? Use + // ServerRoutingInstance#fromServerQueryContext for the above map key. + .sendRequest(rawTableName, asyncQueryResponse, serverRoutingInstance, instanceRequest, requestBytes, timeoutMs); } public void connect(ServerRoutingInstance serverRoutingInstance) @@ -138,6 +134,12 @@ public void connect(ServerRoutingInstance serverRoutingInstance) _serverToChannelMap.computeIfAbsent(serverRoutingInstance, ServerChannel::new).connect(); } + public void connect(ServerQueryRoutingContext serverQueryRoutingContext) + throws InterruptedException, TimeoutException { + _serverToChannelMap.computeIfAbsent(serverQueryRoutingContext.getServerRoutingInstance(), ServerChannel::new) + .connect(); + } + public void shutDown() { // Shut down immediately _eventLoopGroup.shutdownGracefully(0, 0, TimeUnit.SECONDS); @@ -165,26 +167,24 @@ class ServerChannel { _brokerMetrics.setOrUpdateGlobalGauge(BrokerGauge.NETTY_POOLED_CHUNK_SIZE, metric::chunkSize); _bootstrap = new Bootstrap().remoteAddress(serverRoutingInstance.getHostname(), serverRoutingInstance.getPort()) - .option(ChannelOption.ALLOCATOR, bufAllocator) - .group(_eventLoopGroup).channel(_channelClass).option(ChannelOption.SO_KEEPALIVE, true) - .handler(new ChannelInitializer() { + .option(ChannelOption.ALLOCATOR, bufAllocator).group(_eventLoopGroup).channel(_channelClass) + .option(ChannelOption.SO_KEEPALIVE, true).handler(new ChannelInitializer() { @Override protected void initChannel(SocketChannel ch) { if (_tlsConfig != null) { // Add SSL handler first to encrypt and decrypt everything. - ch.pipeline().addLast( - ChannelHandlerFactory.SSL, ChannelHandlerFactory.getClientTlsHandler(_tlsConfig, ch)); + ch.pipeline() + .addLast(ChannelHandlerFactory.SSL, ChannelHandlerFactory.getClientTlsHandler(_tlsConfig, ch)); } ch.pipeline().addLast(ChannelHandlerFactory.getLengthFieldBasedFrameDecoder()); ch.pipeline().addLast(ChannelHandlerFactory.getLengthFieldPrepender()); ch.pipeline().addLast( - ChannelHandlerFactory.getDirectOOMHandler(_queryRouter, _serverRoutingInstance, _serverToChannelMap) - ); + ChannelHandlerFactory.getDirectOOMHandler(_queryRouter, _serverRoutingInstance, _serverToChannelMap)); // NOTE: data table de-serialization happens inside this handler // Revisit if this becomes a bottleneck - ch.pipeline().addLast(ChannelHandlerFactory - .getDataTableHandler(_queryRouter, _serverRoutingInstance, _brokerMetrics)); + ch.pipeline().addLast( + ChannelHandlerFactory.getDataTableHandler(_queryRouter, _serverRoutingInstance, _brokerMetrics)); } }); } @@ -205,12 +205,14 @@ void setSilentShutdown() { } void sendRequest(String rawTableName, AsyncQueryResponse asyncQueryResponse, - ServerRoutingInstance serverRoutingInstance, byte[] requestBytes, long timeoutMs) + ServerRoutingInstance serverRoutingInstance, InstanceRequest instanceRequest, byte[] requestBytes, + long timeoutMs) throws InterruptedException, TimeoutException { if (_channelLock.tryLock(timeoutMs, TimeUnit.MILLISECONDS)) { try { connectWithoutLocking(); - sendRequestWithoutLocking(rawTableName, asyncQueryResponse, serverRoutingInstance, requestBytes); + sendRequestWithoutLocking(rawTableName, asyncQueryResponse, serverRoutingInstance, instanceRequest, + requestBytes); } finally { _channelLock.unlock(); } @@ -230,13 +232,13 @@ void connectWithoutLocking() } void sendRequestWithoutLocking(String rawTableName, AsyncQueryResponse asyncQueryResponse, - ServerRoutingInstance serverRoutingInstance, byte[] requestBytes) { + ServerRoutingInstance serverRoutingInstance, InstanceRequest instanceRequest, byte[] requestBytes) { long startTimeMs = System.currentTimeMillis(); _channel.writeAndFlush(Unpooled.wrappedBuffer(requestBytes)).addListener(f -> { int requestSentLatencyMs = (int) (System.currentTimeMillis() - startTimeMs); _brokerMetrics.addTimedTableValue(rawTableName, BrokerTimer.NETTY_CONNECTION_SEND_REQUEST_LATENCY, requestSentLatencyMs, TimeUnit.MILLISECONDS); - asyncQueryResponse.markRequestSent(serverRoutingInstance, requestSentLatencyMs); + asyncQueryResponse.markRequestSent(serverRoutingInstance, instanceRequest, requestSentLatencyMs); }); _brokerMetrics.addMeteredGlobalValue(BrokerMeter.NETTY_CONNECTION_REQUESTS_SENT, 1); _brokerMetrics.addMeteredGlobalValue(BrokerMeter.NETTY_CONNECTION_BYTES_SENT, requestBytes.length); diff --git a/pinot-core/src/main/java/org/apache/pinot/core/transport/ServerInstance.java b/pinot-core/src/main/java/org/apache/pinot/core/transport/ServerInstance.java index 61a0b73d8a9..d9e8c2024cb 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/transport/ServerInstance.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/transport/ServerInstance.java @@ -23,7 +23,6 @@ import org.apache.commons.lang3.StringUtils; import org.apache.helix.model.InstanceConfig; import org.apache.pinot.common.utils.config.InstanceUtils; -import org.apache.pinot.spi.config.table.TableType; import org.apache.pinot.spi.utils.CommonConstants; import org.apache.pinot.spi.utils.CommonConstants.Helix; @@ -124,27 +123,31 @@ public int getNettyTlsPort() { return _nettyTlsPort; } + public boolean isTlsEnabled() { + return getNettyTlsPort() > 0; + } + // Does not require TLS until all servers guaranteed to be on TLS @Deprecated - public ServerRoutingInstance toServerRoutingInstance(TableType tableType, boolean preferNettyTls) { + public ServerRoutingInstance toServerRoutingInstance(boolean preferNettyTls) { if (preferNettyTls && _nettyTlsPort > 0) { - return new ServerRoutingInstance(_instanceId, _hostname, _nettyTlsPort, tableType, true); + return new ServerRoutingInstance(_instanceId, _hostname, _nettyTlsPort, true); } else { - return new ServerRoutingInstance(_instanceId, _hostname, _port, tableType); + return new ServerRoutingInstance(_instanceId, _hostname, _port); } } - public ServerRoutingInstance toServerRoutingInstance(TableType tableType, RoutingType routingType) { + public ServerRoutingInstance toServerRoutingInstance(RoutingType routingType) { switch (routingType) { case NETTY: Preconditions.checkState(_port > 0, "Netty port is not configured for server: %s", _instanceId); - return new ServerRoutingInstance(_instanceId, _hostname, _port, tableType); + return new ServerRoutingInstance(_instanceId, _hostname, _port); case GRPC: Preconditions.checkState(_grpcPort > 0, "GRPC port is not configured for server: %s", _instanceId); - return new ServerRoutingInstance(_instanceId, _hostname, _grpcPort, tableType); + return new ServerRoutingInstance(_instanceId, _hostname, _grpcPort); case NETTY_TLS: Preconditions.checkState(_nettyTlsPort > 0, "Netty TLS port is not configured for server: %s", _instanceId); - return new ServerRoutingInstance(_instanceId, _hostname, _nettyTlsPort, tableType, true); + return new ServerRoutingInstance(_instanceId, _hostname, _nettyTlsPort, true); default: throw new IllegalStateException("Unsupported routing type: " + routingType); } diff --git a/pinot-core/src/main/java/org/apache/pinot/core/transport/ServerQueryRoutingContext.java b/pinot-core/src/main/java/org/apache/pinot/core/transport/ServerQueryRoutingContext.java new file mode 100644 index 00000000000..ccddf04beb7 --- /dev/null +++ b/pinot-core/src/main/java/org/apache/pinot/core/transport/ServerQueryRoutingContext.java @@ -0,0 +1,69 @@ +/** + * 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.transport; + +import java.util.List; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.pinot.common.request.BrokerRequest; +import org.apache.pinot.spi.config.table.TableType; +import org.apache.pinot.spi.utils.builder.TableNameBuilder; + + +public class ServerQueryRoutingContext { + private final BrokerRequest _brokerRequest; + private final Pair, List> _segmentsToQuery; + private final ServerRoutingInstance _serverRoutingInstance; + private final TableType _tableType; + + public ServerQueryRoutingContext(BrokerRequest brokerRequest, Pair, List> segmentsToQuery, + ServerRoutingInstance serverRoutingInstance) { + // TODO(egalpin): somewhere, should we create a map that uses the BrokerRequest as the key and combines all + // segmentsToQuery? this would allow servers to combine results from segments that share a common query before + // sending back to broker + _brokerRequest = brokerRequest; + _segmentsToQuery = segmentsToQuery; + _serverRoutingInstance = serverRoutingInstance; + String tableName = _brokerRequest.getPinotQuery().getDataSource().getTableName(); + _tableType = TableNameBuilder.getTableTypeFromTableName(tableName); + } + + public BrokerRequest getBrokerRequest() { + return _brokerRequest; + } + + public Pair, List> getSegmentsToQuery() { + return _segmentsToQuery; + } + + public List getRequiredSegmentsToQuery() { + return getSegmentsToQuery().getLeft(); + } + + public List getOptionalSegmentsToQuery() { + return getSegmentsToQuery().getRight(); + } + + public TableType getTableType() { + return _tableType; + } + + public ServerRoutingInstance getServerRoutingInstance() { + return _serverRoutingInstance; + } +} diff --git a/pinot-core/src/main/java/org/apache/pinot/core/transport/ServerRoutingInstance.java b/pinot-core/src/main/java/org/apache/pinot/core/transport/ServerRoutingInstance.java index faa2cd91a34..5c9dd4904c9 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/transport/ServerRoutingInstance.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/transport/ServerRoutingInstance.java @@ -23,43 +23,36 @@ import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import javax.annotation.concurrent.ThreadSafe; -import org.apache.pinot.spi.config.table.TableType; import org.apache.pinot.spi.utils.CommonConstants.Helix; /** * The {@code ServerRoutingInstance} class represents the routing target instance which contains the information of - * hostname, port, and table type it serves. - *

Different table types on same host and port are counted as different instances. Therefore, one single Pinot Server - * might be treated as two different routing target instances based on the types of table it serves. + * hostname and port. */ @ThreadSafe public final class ServerRoutingInstance { - private static final String SHORT_OFFLINE_SUFFIX = "_O"; - private static final String SHORT_REALTIME_SUFFIX = "_R"; private static final Map SHORT_HOSTNAME_MAP = new ConcurrentHashMap<>(); private final String _instanceId; private final String _hostname; private final int _port; - private final TableType _tableType; private final boolean _tlsEnabled; - public ServerRoutingInstance(String instanceId, String hostname, int port, TableType tableType, boolean tlsEnabled) { + public ServerRoutingInstance(String instanceId, String hostname, int port, boolean tlsEnabled) { _instanceId = instanceId; _hostname = hostname; _port = port; - _tableType = tableType; _tlsEnabled = tlsEnabled; } - public ServerRoutingInstance(String instanceId, String hostname, int port, TableType tableType) { - this(instanceId, hostname, port, tableType, false); + public ServerRoutingInstance(String instanceId, String hostname, int port) { + this(instanceId, hostname, port, false); } @VisibleForTesting - public ServerRoutingInstance(String hostname, int port, TableType tableType) { - this(Helix.PREFIX_OF_SERVER_INSTANCE + hostname + "_" + port, hostname, port, tableType); + public ServerRoutingInstance(String hostname, int port) { + this(Helix.PREFIX_OF_SERVER_INSTANCE + hostname + "_" + port, hostname, port); } public String getInstanceId() { @@ -74,20 +67,17 @@ public int getPort() { return _port; } - public TableType getTableType() { - return _tableType; + public static String toShortName(String hostname) { + try { + InternetDomainName domainName = InternetDomainName.from(hostname); + return domainName.parts().get(0); + } catch (Exception e) { + return hostname; + } } public String getShortName() { - String shortHostname = SHORT_HOSTNAME_MAP.computeIfAbsent(_hostname, hostname -> { - try { - InternetDomainName domainName = InternetDomainName.from(hostname); - return domainName.parts().get(0); - } catch (Exception e) { - return hostname; - } - }); - return shortHostname + (_tableType == TableType.OFFLINE ? SHORT_OFFLINE_SUFFIX : SHORT_REALTIME_SUFFIX); + return SHORT_HOSTNAME_MAP.computeIfAbsent(_hostname, ServerRoutingInstance::toShortName); } public boolean isTlsEnabled() { @@ -103,16 +93,16 @@ public boolean equals(Object o) { return false; } ServerRoutingInstance that = (ServerRoutingInstance) o; - // NOTE: Only check hostname, port and tableType for performance concern because they can identify a routing + // NOTE: Only check hostname and port for performance concern because they can identify a routing // instance within the same query - return _hostname.equals(that._hostname) && _port == that._port && _tableType == that._tableType; + return _hostname.equals(that._hostname) && _port == that._port; } @Override public int hashCode() { - // NOTE: Only check hostname, port and tableType for performance concern because they can identify a routing + // NOTE: Only check hostname and port for performance concern because they can identify a routing // instance within the same query - return 31 * 31 * _hostname.hashCode() + 31 * Integer.hashCode(_port) + _tableType.hashCode(); + return 31 * _hostname.hashCode() + Integer.hashCode(_port); } @Override diff --git a/pinot-core/src/test/java/org/apache/pinot/core/query/reduce/BrokerReduceServiceTest.java b/pinot-core/src/test/java/org/apache/pinot/core/query/reduce/BrokerReduceServiceTest.java index ed08918251a..9513d92a273 100644 --- a/pinot-core/src/test/java/org/apache/pinot/core/query/reduce/BrokerReduceServiceTest.java +++ b/pinot-core/src/test/java/org/apache/pinot/core/query/reduce/BrokerReduceServiceTest.java @@ -19,6 +19,8 @@ package org.apache.pinot.core.query.reduce; import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -33,7 +35,6 @@ import org.apache.pinot.core.common.datatable.DataTableBuilder; import org.apache.pinot.core.common.datatable.DataTableBuilderFactory; import org.apache.pinot.core.transport.ServerRoutingInstance; -import org.apache.pinot.spi.config.table.TableType; import org.apache.pinot.spi.env.PinotConfiguration; import org.apache.pinot.spi.utils.CommonConstants.Broker; import org.apache.pinot.sql.parsers.CalciteSqlCompiler; @@ -63,11 +64,11 @@ public void testReduceTimeout() dataTableBuilder.finishRow(); } DataTable dataTable = dataTableBuilder.build(); - Map dataTableMap = new HashMap<>(); + Map> dataTableMap = new HashMap<>(); int numInstances = 1000; for (int i = 0; i < numInstances; i++) { - ServerRoutingInstance instance = new ServerRoutingInstance("localhost", i, TableType.OFFLINE); - dataTableMap.put(instance, dataTable); + ServerRoutingInstance instance = new ServerRoutingInstance("localhost", i); + dataTableMap.computeIfAbsent(instance, k -> new ArrayList<>()).add(dataTable); } long reduceTimeoutMs = 1; BrokerResponseNative brokerResponse = diff --git a/pinot-core/src/test/java/org/apache/pinot/core/query/reduce/StreamingReduceServiceTest.java b/pinot-core/src/test/java/org/apache/pinot/core/query/reduce/StreamingReduceServiceTest.java index a48b12563d1..306d53c9075 100644 --- a/pinot-core/src/test/java/org/apache/pinot/core/query/reduce/StreamingReduceServiceTest.java +++ b/pinot-core/src/test/java/org/apache/pinot/core/query/reduce/StreamingReduceServiceTest.java @@ -27,7 +27,6 @@ import java.util.function.Predicate; import org.apache.pinot.common.proto.Server; import org.apache.pinot.core.transport.ServerRoutingInstance; -import org.apache.pinot.spi.config.table.TableType; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; import org.testng.annotations.Test; @@ -48,7 +47,7 @@ public void testThreadExceptionTransfer() { RuntimeException innerException = new RuntimeException(exceptionMessage); when(mockedResponse.next()).thenThrow(innerException); ExecutorService threadPoolService = Executors.newFixedThreadPool(1); - ServerRoutingInstance routingInstance = new ServerRoutingInstance("localhost", 9527, TableType.OFFLINE); + ServerRoutingInstance routingInstance = new ServerRoutingInstance("localhost", 9527); // supposedly we can use TestNG's annotation like @Test(expectedExceptions = { IOException.class }) to verify // here we hope to verify deeper to make sure the thrown exception is nested inside the exception assertTrue(verifyException(() -> { @@ -77,7 +76,7 @@ public Void answer(InvocationOnMock invocationOnMock) } }); final ExecutorService threadPoolService = Executors.newFixedThreadPool(1); - final ServerRoutingInstance routingInstance = new ServerRoutingInstance("localhost", 9527, TableType.OFFLINE); + final ServerRoutingInstance routingInstance = new ServerRoutingInstance("localhost", 9527); //We cannot use TestNG's annotation like @Test(expectedExceptions = { IOException.class }) to verify // because the Exception we hope to verify is nested inside the final exception. assertTrue(verifyException(() -> { diff --git a/pinot-core/src/test/java/org/apache/pinot/core/transport/QueryRoutingTest.java b/pinot-core/src/test/java/org/apache/pinot/core/transport/QueryRoutingTest.java index cec413e4248..d0b3179e4a6 100644 --- a/pinot-core/src/test/java/org/apache/pinot/core/transport/QueryRoutingTest.java +++ b/pinot-core/src/test/java/org/apache/pinot/core/transport/QueryRoutingTest.java @@ -20,6 +20,7 @@ import com.google.common.util.concurrent.Futures; import java.io.IOException; +import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -33,12 +34,13 @@ import org.apache.pinot.common.utils.DataSchema; import org.apache.pinot.core.common.datatable.DataTableBuilder; import org.apache.pinot.core.common.datatable.DataTableBuilderFactory; +import org.apache.pinot.core.query.request.ServerQueryRequest; import org.apache.pinot.core.query.scheduler.QueryScheduler; import org.apache.pinot.core.transport.server.routing.stats.ServerRoutingStatsManager; import org.apache.pinot.server.access.AccessControl; -import org.apache.pinot.spi.config.table.TableType; import org.apache.pinot.spi.env.PinotConfiguration; import org.apache.pinot.spi.utils.CommonConstants; +import org.apache.pinot.spi.utils.builder.TableNameBuilder; import org.apache.pinot.sql.parsers.CalciteSqlCompiler; import org.apache.pinot.util.TestUtils; import org.testng.annotations.AfterClass; @@ -58,14 +60,26 @@ public class QueryRoutingTest { private static final int TEST_PORT = 12345; private static final ServerInstance SERVER_INSTANCE = new ServerInstance("localhost", TEST_PORT); - private static final ServerRoutingInstance OFFLINE_SERVER_ROUTING_INSTANCE = - SERVER_INSTANCE.toServerRoutingInstance(TableType.OFFLINE, ServerInstance.RoutingType.NETTY); - private static final ServerRoutingInstance REALTIME_SERVER_ROUTING_INSTANCE = - SERVER_INSTANCE.toServerRoutingInstance(TableType.REALTIME, ServerInstance.RoutingType.NETTY); - private static final BrokerRequest BROKER_REQUEST = - CalciteSqlCompiler.compileToBrokerRequest("SELECT * FROM testTable"); - private static final Map, List>> ROUTING_TABLE = - Collections.singletonMap(SERVER_INSTANCE, Pair.of(Collections.emptyList(), Collections.emptyList())); + private static final ServerRoutingInstance SERVER_ROUTING_INSTANCE = + SERVER_INSTANCE.toServerRoutingInstance(ServerInstance.RoutingType.NETTY); + private static final String TABLE_NAME = "testTable"; + private static final BrokerRequest OFFLINE_BROKER_REQUEST = CalciteSqlCompiler.compileToBrokerRequest( + "SELECT * FROM " + TableNameBuilder.OFFLINE.tableNameWithType(TABLE_NAME)); + private static final BrokerRequest REALTIME_BROKER_REQUEST = CalciteSqlCompiler.compileToBrokerRequest( + "SELECT * FROM " + TableNameBuilder.REALTIME.tableNameWithType(TABLE_NAME)); + + private static final ServerQueryRoutingContext OFFLINE_BROKER_REQ_CONTEXT = + new ServerQueryRoutingContext(OFFLINE_BROKER_REQUEST, Pair.of(Collections.emptyList(), Collections.emptyList()), + SERVER_ROUTING_INSTANCE); + private static final ServerQueryRoutingContext REALTIME_BROKER_REQ_CONTEXT = + new ServerQueryRoutingContext(REALTIME_BROKER_REQUEST, Pair.of(Collections.emptyList(), Collections.emptyList()), + SERVER_ROUTING_INSTANCE); + private static final Map> OFFLINE_ROUTING_TABLE = + Collections.singletonMap(SERVER_INSTANCE, List.of(OFFLINE_BROKER_REQ_CONTEXT)); + private static final Map> REALTIME_ROUTING_TABLE = + Collections.singletonMap(SERVER_INSTANCE, List.of(REALTIME_BROKER_REQ_CONTEXT)); + private static final Map> HYBRID_ROUTING_TABLE = + Collections.singletonMap(SERVER_INSTANCE, List.of(OFFLINE_BROKER_REQ_CONTEXT, REALTIME_BROKER_REQ_CONTEXT)); private QueryRouter _queryRouter; private ServerRoutingStatsManager _serverRoutingStatsManager; @@ -88,17 +102,27 @@ void deregisterServerMetrics() { } private QueryServer getQueryServer(int responseDelayMs, byte[] responseBytes) { - return getQueryServer(responseDelayMs, responseBytes, TEST_PORT); + ServerMetrics serverMetrics = mock(ServerMetrics.class); + InstanceRequestHandler handler = new InstanceRequestHandler("server01", new PinotConfiguration(), + mockQueryScheduler(responseDelayMs, responseBytes), serverMetrics, mock(AccessControl.class)); + ServerMetrics.register(serverMetrics); + return new QueryServer(TEST_PORT, null, handler); } - private QueryServer getQueryServer(int responseDelayMs, byte[] responseBytes, int port) { + private QueryServer getQueryServer(int responseDelayMs, DataTable offlineDataTable, DataTable realtimeDataTable, + int port) { ServerMetrics serverMetrics = mock(ServerMetrics.class); InstanceRequestHandler handler = new InstanceRequestHandler("server01", new PinotConfiguration(), - mockQueryScheduler(responseDelayMs, responseBytes), serverMetrics, mock(AccessControl.class)); + mockQueryScheduler(responseDelayMs, offlineDataTable, realtimeDataTable), serverMetrics, + mock(AccessControl.class)); ServerMetrics.register(serverMetrics); return new QueryServer(port, null, handler); } + private QueryServer getQueryServer(int responseDelayMs, DataTable offlineDataTable, DataTable realtimeDataTable) { + return getQueryServer(responseDelayMs, offlineDataTable, realtimeDataTable, TEST_PORT); + } + private QueryScheduler mockQueryScheduler(int responseDelayMs, byte[] responseBytes) { QueryScheduler queryScheduler = mock(QueryScheduler.class); when(queryScheduler.submit(any())).thenAnswer(invocation -> { @@ -108,60 +132,93 @@ private QueryScheduler mockQueryScheduler(int responseDelayMs, byte[] responseBy return queryScheduler; } + private QueryScheduler mockQueryScheduler(int responseDelayMs, DataTable offlineDataTable, + DataTable realtimeDataTable) { + QueryScheduler queryScheduler = mock(QueryScheduler.class); + when(queryScheduler.submit(any())).thenAnswer(invocation -> { + Thread.sleep(responseDelayMs); + String queryHash = String.valueOf(((ServerQueryRequest) invocation.getArguments()[0]).getQueryHash()); + if (queryHash.equals(realtimeDataTable.getMetadata().get(MetadataKey.QUERY_HASH.getName()))) { + return Futures.immediateFuture(realtimeDataTable.toBytes()); + } else if (queryHash.equals(offlineDataTable.getMetadata().get(MetadataKey.QUERY_HASH.getName()))) { + return Futures.immediateFuture(offlineDataTable.toBytes()); + } + return Futures.immediateFuture(new byte[0]); + }); + return queryScheduler; + } + @Test public void testValidResponse() throws Exception { long requestId = 123; - DataTable dataTable = DataTableBuilderFactory.getEmptyDataTable(); - dataTable.getMetadata().put(MetadataKey.REQUEST_ID.getName(), Long.toString(requestId)); - byte[] responseBytes = dataTable.toBytes(); + DataTable offlineDataTable = DataTableBuilderFactory.getEmptyDataTable(); + offlineDataTable.getMetadata().put(MetadataKey.REQUEST_ID.getName(), Long.toString(requestId)); + offlineDataTable.getMetadata() + .put(MetadataKey.QUERY_HASH.getName(), Integer.toString(OFFLINE_BROKER_REQUEST.getPinotQuery().hashCode())); + byte[] offlineResponseBytes = offlineDataTable.toBytes(); + + DataTable realtimeDataTable = DataTableBuilderFactory.getEmptyDataTable(); + realtimeDataTable.getMetadata().put(MetadataKey.REQUEST_ID.getName(), Long.toString(requestId)); + realtimeDataTable.getMetadata() + .put(MetadataKey.QUERY_HASH.getName(), Integer.toString(REALTIME_BROKER_REQUEST.getPinotQuery().hashCode())); + byte[] realtimeResponseBytes = realtimeDataTable.toBytes(); + String serverId = SERVER_INSTANCE.getInstanceId(); // Start the server - QueryServer queryServer = getQueryServer(0, responseBytes); + QueryServer queryServer = getQueryServer(0, offlineDataTable, realtimeDataTable); queryServer.start(); // OFFLINE only AsyncQueryResponse asyncQueryResponse = - _queryRouter.submitQuery(requestId, "testTable", BROKER_REQUEST, ROUTING_TABLE, null, null, 600_000L); - Map response = asyncQueryResponse.getFinalResponses(); + _queryRouter.submitQuery(requestId, "testTable", OFFLINE_ROUTING_TABLE, 600_000L); + Map> response = asyncQueryResponse.getFinalResponses(); assertEquals(response.size(), 1); - assertTrue(response.containsKey(OFFLINE_SERVER_ROUTING_INSTANCE)); - ServerResponse serverResponse = response.get(OFFLINE_SERVER_ROUTING_INSTANCE); + assertTrue(response.containsKey(SERVER_ROUTING_INSTANCE)); + assertEquals(response.get(SERVER_ROUTING_INSTANCE).size(), 1); + ServerResponse serverResponse = new ArrayList<>(response.get(SERVER_ROUTING_INSTANCE).values()).get(0); assertNotNull(serverResponse.getDataTable()); - assertEquals(serverResponse.getResponseSize(), responseBytes.length); + assertEquals(serverResponse.getResponseSize(), offlineResponseBytes.length); // 2 requests - query submit and query response. _requestCount += 2; waitForStatsUpdate(_requestCount); assertEquals(_serverRoutingStatsManager.fetchNumInFlightRequestsForServer(serverId).intValue(), 0); // REALTIME only - asyncQueryResponse = - _queryRouter.submitQuery(requestId, "testTable", null, null, BROKER_REQUEST, ROUTING_TABLE, 1_000L); + asyncQueryResponse = _queryRouter.submitQuery(requestId, "testTable", REALTIME_ROUTING_TABLE, 1_000L); response = asyncQueryResponse.getFinalResponses(); assertEquals(response.size(), 1); - assertTrue(response.containsKey(REALTIME_SERVER_ROUTING_INSTANCE)); - serverResponse = response.get(REALTIME_SERVER_ROUTING_INSTANCE); + assertTrue(response.containsKey(SERVER_ROUTING_INSTANCE)); + assertEquals(response.get(SERVER_ROUTING_INSTANCE).size(), 1); + serverResponse = new ArrayList<>(response.get(SERVER_ROUTING_INSTANCE).values()).get(0); assertNotNull(serverResponse.getDataTable()); - assertEquals(serverResponse.getResponseSize(), responseBytes.length); + assertEquals(serverResponse.getResponseSize(), realtimeResponseBytes.length); _requestCount += 2; waitForStatsUpdate(_requestCount); assertEquals(_serverRoutingStatsManager.fetchNumInFlightRequestsForServer(serverId).intValue(), 0); // Hybrid - asyncQueryResponse = - _queryRouter.submitQuery(requestId, "testTable", BROKER_REQUEST, ROUTING_TABLE, BROKER_REQUEST, ROUTING_TABLE, - 1_000L); + asyncQueryResponse = _queryRouter.submitQuery(requestId, "testTable", HYBRID_ROUTING_TABLE, 1_000L); response = asyncQueryResponse.getFinalResponses(); - assertEquals(response.size(), 2); - assertTrue(response.containsKey(OFFLINE_SERVER_ROUTING_INSTANCE)); - serverResponse = response.get(OFFLINE_SERVER_ROUTING_INSTANCE); - assertNotNull(serverResponse.getDataTable()); - assertEquals(serverResponse.getResponseSize(), responseBytes.length); - assertTrue(response.containsKey(REALTIME_SERVER_ROUTING_INSTANCE)); - serverResponse = response.get(REALTIME_SERVER_ROUTING_INSTANCE); - assertNotNull(serverResponse.getDataTable()); - assertEquals(serverResponse.getResponseSize(), responseBytes.length); + assertEquals(response.size(), 1); + assertTrue(response.containsKey(SERVER_ROUTING_INSTANCE)); + assertEquals(response.get(SERVER_ROUTING_INSTANCE).size(), 2); + + int accountedFor = 0; + for (ServerResponse serverResponse1 : response.get(SERVER_ROUTING_INSTANCE).values()) { + assertNotNull(serverResponse1.getDataTable()); + if (serverResponse1.getDataTable().getMetadata().get(MetadataKey.QUERY_HASH.getName()) + .equals(offlineDataTable.getMetadata().get(MetadataKey.QUERY_HASH.getName()))) { + assertEquals(serverResponse1.getResponseSize(), offlineResponseBytes.length); + accountedFor++; + } else if (serverResponse1.getDataTable().getMetadata().get(MetadataKey.QUERY_HASH.getName()) + .equals(realtimeDataTable.getMetadata().get(MetadataKey.QUERY_HASH.getName()))) { + assertEquals(serverResponse1.getResponseSize(), realtimeResponseBytes.length); + accountedFor++; + } + } + assertEquals(accountedFor, 2, "Hybrid should have created 1 realtime and 1 offline request/response"); _requestCount += 4; waitForStatsUpdate(_requestCount); assertEquals(_serverRoutingStatsManager.fetchNumInFlightRequestsForServer(serverId).intValue(), 0); @@ -182,11 +239,12 @@ public void testInvalidResponse() long startTimeMs = System.currentTimeMillis(); AsyncQueryResponse asyncQueryResponse = - _queryRouter.submitQuery(requestId, "testTable", BROKER_REQUEST, ROUTING_TABLE, null, null, 1_000L); - Map response = asyncQueryResponse.getFinalResponses(); + _queryRouter.submitQuery(requestId, "testTable", OFFLINE_ROUTING_TABLE, 1_000L); + Map> response = asyncQueryResponse.getFinalResponses(); assertEquals(response.size(), 1); - assertTrue(response.containsKey(OFFLINE_SERVER_ROUTING_INSTANCE)); - ServerResponse serverResponse = response.get(OFFLINE_SERVER_ROUTING_INSTANCE); + assertTrue(response.containsKey(SERVER_ROUTING_INSTANCE)); + assertEquals(response.get(SERVER_ROUTING_INSTANCE).size(), 1); + ServerResponse serverResponse = new ArrayList<>(response.get(SERVER_ROUTING_INSTANCE).values()).get(0); assertNull(serverResponse.getDataTable()); assertEquals(serverResponse.getResponseDelayMs(), -1); assertEquals(serverResponse.getResponseSize(), 0); @@ -205,22 +263,29 @@ public void testInvalidResponse() public void testNonMatchingRequestId() throws Exception { long requestId = 123; - DataTable dataTable = DataTableBuilderFactory.getEmptyDataTable(); - dataTable.getMetadata().put(MetadataKey.REQUEST_ID.getName(), Long.toString(requestId)); - byte[] responseBytes = dataTable.toBytes(); + DataTable offlineDataTable = DataTableBuilderFactory.getEmptyDataTable(); + offlineDataTable.getMetadata().put(MetadataKey.REQUEST_ID.getName(), Long.toString(requestId)); + offlineDataTable.getMetadata() + .put(MetadataKey.QUERY_HASH.getName(), Integer.toString(OFFLINE_BROKER_REQUEST.getPinotQuery().hashCode())); + + DataTable realtimeDataTable = DataTableBuilderFactory.getEmptyDataTable(); + realtimeDataTable.getMetadata().put(MetadataKey.REQUEST_ID.getName(), Long.toString(requestId)); + realtimeDataTable.getMetadata() + .put(MetadataKey.QUERY_HASH.getName(), Integer.toString(REALTIME_BROKER_REQUEST.getPinotQuery().hashCode())); String serverId = SERVER_INSTANCE.getInstanceId(); // Start the server - QueryServer queryServer = getQueryServer(0, responseBytes); + QueryServer queryServer = getQueryServer(0, offlineDataTable, realtimeDataTable); queryServer.start(); long startTimeMs = System.currentTimeMillis(); AsyncQueryResponse asyncQueryResponse = - _queryRouter.submitQuery(requestId + 1, "testTable", BROKER_REQUEST, ROUTING_TABLE, null, null, 1_000L); - Map response = asyncQueryResponse.getFinalResponses(); + _queryRouter.submitQuery(requestId + 1, "testTable", OFFLINE_ROUTING_TABLE, 1_000L); + Map> response = asyncQueryResponse.getFinalResponses(); assertEquals(response.size(), 1); - assertTrue(response.containsKey(OFFLINE_SERVER_ROUTING_INSTANCE)); - ServerResponse serverResponse = response.get(OFFLINE_SERVER_ROUTING_INSTANCE); + assertTrue(response.containsKey(SERVER_ROUTING_INSTANCE)); + assertEquals(response.get(SERVER_ROUTING_INSTANCE).size(), 1); + ServerResponse serverResponse = new ArrayList<>(response.get(SERVER_ROUTING_INSTANCE).values()).get(0); assertNull(serverResponse.getDataTable()); assertEquals(serverResponse.getResponseDelayMs(), -1); assertEquals(serverResponse.getResponseSize(), 0); @@ -242,26 +307,34 @@ public void testServerDown() // To avoid flakyness, set timeoutMs to 2000 msec. For some test runs, it can take up to // 1400 msec to mark request as failed. long timeoutMs = 2000L; - DataTable dataTable = DataTableBuilderFactory.getEmptyDataTable(); - dataTable.getMetadata().put(MetadataKey.REQUEST_ID.getName(), Long.toString(requestId)); - byte[] responseBytes = dataTable.toBytes(); + DataTable offlineDataTable = DataTableBuilderFactory.getEmptyDataTable(); + offlineDataTable.getMetadata().put(MetadataKey.REQUEST_ID.getName(), Long.toString(requestId)); + offlineDataTable.getMetadata() + .put(MetadataKey.QUERY_HASH.getName(), Integer.toString(OFFLINE_BROKER_REQUEST.getPinotQuery().hashCode())); + + DataTable realtimeDataTable = DataTableBuilderFactory.getEmptyDataTable(); + realtimeDataTable.getMetadata().put(MetadataKey.REQUEST_ID.getName(), Long.toString(requestId)); + realtimeDataTable.getMetadata() + .put(MetadataKey.QUERY_HASH.getName(), Integer.toString(REALTIME_BROKER_REQUEST.getPinotQuery().hashCode())); + String serverId = SERVER_INSTANCE.getInstanceId(); // Start the server - QueryServer queryServer = getQueryServer(500, responseBytes); + QueryServer queryServer = getQueryServer(500, offlineDataTable, realtimeDataTable); queryServer.start(); long startTimeMs = System.currentTimeMillis(); AsyncQueryResponse asyncQueryResponse = - _queryRouter.submitQuery(requestId + 1, "testTable", BROKER_REQUEST, ROUTING_TABLE, null, null, timeoutMs); + _queryRouter.submitQuery(requestId + 1, "testTable", OFFLINE_ROUTING_TABLE, timeoutMs); // Shut down the server before getting the response queryServer.shutDown(); - Map response = asyncQueryResponse.getFinalResponses(); + Map> response = asyncQueryResponse.getFinalResponses(); assertEquals(response.size(), 1); - assertTrue(response.containsKey(OFFLINE_SERVER_ROUTING_INSTANCE)); - ServerResponse serverResponse = response.get(OFFLINE_SERVER_ROUTING_INSTANCE); + assertTrue(response.containsKey(SERVER_ROUTING_INSTANCE)); + assertEquals(response.get(SERVER_ROUTING_INSTANCE).size(), 1); + ServerResponse serverResponse = new ArrayList<>(response.get(SERVER_ROUTING_INSTANCE).values()).get(0); assertNull(serverResponse.getDataTable()); assertEquals(serverResponse.getResponseDelayMs(), -1); assertEquals(serverResponse.getResponseSize(), 0); @@ -272,15 +345,14 @@ public void testServerDown() waitForStatsUpdate(_requestCount); assertEquals(_serverRoutingStatsManager.fetchNumInFlightRequestsForServer(serverId).intValue(), 0); - // Submit query after server is down startTimeMs = System.currentTimeMillis(); - asyncQueryResponse = - _queryRouter.submitQuery(requestId + 1, "testTable", BROKER_REQUEST, ROUTING_TABLE, null, null, timeoutMs); + asyncQueryResponse = _queryRouter.submitQuery(requestId + 1, "testTable", OFFLINE_ROUTING_TABLE, timeoutMs); response = asyncQueryResponse.getFinalResponses(); assertEquals(response.size(), 1); - assertTrue(response.containsKey(OFFLINE_SERVER_ROUTING_INSTANCE)); - serverResponse = response.get(OFFLINE_SERVER_ROUTING_INSTANCE); + assertTrue(response.containsKey(SERVER_ROUTING_INSTANCE)); + assertEquals(response.get(SERVER_ROUTING_INSTANCE).size(), 1); + serverResponse = new ArrayList<>(response.get(SERVER_ROUTING_INSTANCE).values()).get(0); assertNull(serverResponse.getDataTable()); assertEquals(serverResponse.getSubmitDelayMs(), -1); assertEquals(serverResponse.getResponseDelayMs(), -1); @@ -302,12 +374,25 @@ public void testSkipUnavailableServer() ServerInstance serverInstance1 = new ServerInstance("localhost", port); ServerInstance serverInstance2 = new ServerInstance("localhost", port + 1); ServerRoutingInstance serverRoutingInstance1 = - serverInstance1.toServerRoutingInstance(TableType.OFFLINE, ServerInstance.RoutingType.NETTY); + serverInstance1.toServerRoutingInstance(ServerInstance.RoutingType.NETTY); ServerRoutingInstance serverRoutingInstance2 = - serverInstance2.toServerRoutingInstance(TableType.OFFLINE, ServerInstance.RoutingType.NETTY); - Map, List>> routingTable = - Map.of(serverInstance1, Pair.of(Collections.emptyList(), Collections.emptyList()), serverInstance2, - Pair.of(Collections.emptyList(), Collections.emptyList())); + serverInstance2.toServerRoutingInstance(ServerInstance.RoutingType.NETTY); +// Map, List>> routingTable = +// Map.of(serverInstance1, Pair.of(Collections.emptyList(), Collections.emptyList()), serverInstance2, +// Pair.of(Collections.emptyList(), Collections.emptyList())); + + BrokerRequest brokerRequest = + CalciteSqlCompiler.compileToBrokerRequest("SET skipUnavailableServers=true; SELECT * FROM testTable_OFFLINE"); + + ServerQueryRoutingContext offlineBrokerRequestContext1 = + new ServerQueryRoutingContext(brokerRequest, Pair.of(Collections.emptyList(), Collections.emptyList()), + serverRoutingInstance1); + ServerQueryRoutingContext offlineBrokerRequestContext2 = + new ServerQueryRoutingContext(brokerRequest, Pair.of(Collections.emptyList(), Collections.emptyList()), + serverRoutingInstance2); + Map> routingTable = + Map.of(serverInstance1, List.of(offlineBrokerRequestContext1), serverInstance2, + List.of(offlineBrokerRequestContext2)); long requestId = 123; DataSchema dataSchema = @@ -319,25 +404,22 @@ public void testSkipUnavailableServer() DataTable dataTableSuccess = builder.build(); Map dataTableMetadata = dataTableSuccess.getMetadata(); dataTableMetadata.put(MetadataKey.REQUEST_ID.getName(), Long.toString(requestId)); - byte[] successResponseBytes = dataTableSuccess.toBytes(); + dataTableMetadata.put(MetadataKey.QUERY_HASH.getName(), Integer.toString(brokerRequest.getPinotQuery().hashCode())); // Only start a single QueryServer, on port from serverInstance1 - QueryServer queryServer = getQueryServer(500, successResponseBytes, port); + QueryServer queryServer = getQueryServer(500, dataTableSuccess, dataTableSuccess, port); queryServer.start(); // Submit the query with skipUnavailableServers=true, the single started server should return a valid response - BrokerRequest brokerRequest = - CalciteSqlCompiler.compileToBrokerRequest("SET skipUnavailableServers=true; SELECT * FROM testTable"); long startTime = System.currentTimeMillis(); - AsyncQueryResponse asyncQueryResponse = - _queryRouter.submitQuery(requestId, "testTable", brokerRequest, routingTable, null, null, 10_000L); - Map response = asyncQueryResponse.getFinalResponses(); + AsyncQueryResponse asyncQueryResponse = _queryRouter.submitQuery(requestId, "testTable", routingTable, 10_000L); + Map> response = asyncQueryResponse.getFinalResponses(); assertEquals(response.size(), 2); assertTrue(response.containsKey(serverRoutingInstance1)); assertTrue(response.containsKey(serverRoutingInstance2)); - ServerResponse serverResponse1 = response.get(serverRoutingInstance1); - ServerResponse serverResponse2 = response.get(serverRoutingInstance2); + ServerResponse serverResponse1 = new ArrayList<>(response.get(serverRoutingInstance1).values()).get(0); + ServerResponse serverResponse2 = new ArrayList<>(response.get(serverRoutingInstance2).values()).get(0); assertNotNull(serverResponse1.getDataTable()); assertNull(serverResponse2.getDataTable()); assertTrue(serverResponse1.getResponseDelayMs() > 500); // > response delay set by getQueryServer @@ -350,18 +432,34 @@ public void testSkipUnavailableServer() assertEquals( _serverRoutingStatsManager.fetchNumInFlightRequestsForServer(serverInstance2.getInstanceId()).intValue(), 0); + queryServer.shutDown(); + // Submit the same query without skipUnavailableServers, the servers should not return any response - brokerRequest = CalciteSqlCompiler.compileToBrokerRequest("SELECT * FROM testTable"); + brokerRequest = CalciteSqlCompiler.compileToBrokerRequest("SELECT * FROM testTable_OFFLINE"); + dataTableMetadata.put(MetadataKey.QUERY_HASH.getName(), Integer.toString(brokerRequest.getPinotQuery().hashCode())); + + offlineBrokerRequestContext1 = + new ServerQueryRoutingContext(brokerRequest, Pair.of(Collections.emptyList(), Collections.emptyList()), + serverRoutingInstance1); + offlineBrokerRequestContext2 = + new ServerQueryRoutingContext(brokerRequest, Pair.of(Collections.emptyList(), Collections.emptyList()), + serverRoutingInstance2); + routingTable = Map.of(serverInstance1, List.of(offlineBrokerRequestContext1), serverInstance2, + List.of(offlineBrokerRequestContext2)); + + // Start a new query server with updated data table + queryServer = getQueryServer(500, dataTableSuccess, dataTableSuccess, port); + queryServer.start(); startTime = System.currentTimeMillis(); - asyncQueryResponse = - _queryRouter.submitQuery(requestId, "testTable", brokerRequest, routingTable, null, null, 10_000L); + + asyncQueryResponse = _queryRouter.submitQuery(requestId, "testTable", routingTable, 10_000L); response = asyncQueryResponse.getFinalResponses(); assertEquals(response.size(), 2); assertTrue(response.containsKey(serverRoutingInstance1)); assertTrue(response.containsKey(serverRoutingInstance2)); - serverResponse1 = response.get(serverRoutingInstance1); - serverResponse2 = response.get(serverRoutingInstance2); + serverResponse1 = new ArrayList<>(response.get(serverRoutingInstance1).values()).get(0); + serverResponse2 = new ArrayList<>(response.get(serverRoutingInstance2).values()).get(0); assertNull(serverResponse1.getDataTable()); assertNull(serverResponse2.getDataTable()); assertTrue(serverResponse1.getResponseDelayMs() < 100); diff --git a/pinot-core/src/test/java/org/apache/pinot/core/transport/ServerChannelsTest.java b/pinot-core/src/test/java/org/apache/pinot/core/transport/ServerChannelsTest.java index abbbe5be56c..6f57af374c0 100644 --- a/pinot-core/src/test/java/org/apache/pinot/core/transport/ServerChannelsTest.java +++ b/pinot-core/src/test/java/org/apache/pinot/core/transport/ServerChannelsTest.java @@ -24,7 +24,6 @@ import org.apache.pinot.common.metrics.BrokerMetrics; import org.apache.pinot.common.request.BrokerRequest; import org.apache.pinot.common.request.InstanceRequest; -import org.apache.pinot.spi.config.table.TableType; import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; import org.testng.annotations.DataProvider; @@ -68,7 +67,7 @@ public void testConnect(boolean nativeTransportEnabled) QueryRouter queryRouter = mock(QueryRouter.class); ServerRoutingInstance serverRoutingInstance = - new ServerRoutingInstance("localhost", _dummyServer.getAddress().getPort(), TableType.REALTIME); + new ServerRoutingInstance("localhost", _dummyServer.getAddress().getPort()); ServerChannels serverChannels = new ServerChannels(queryRouter, brokerMetrics, nettyConfig, null); serverChannels.connect(serverRoutingInstance); diff --git a/pinot-core/src/test/java/org/apache/pinot/queries/BaseQueriesTest.java b/pinot-core/src/test/java/org/apache/pinot/queries/BaseQueriesTest.java index 72af8ab860c..ad9eed8f757 100644 --- a/pinot-core/src/test/java/org/apache/pinot/queries/BaseQueriesTest.java +++ b/pinot-core/src/test/java/org/apache/pinot/queries/BaseQueriesTest.java @@ -20,6 +20,7 @@ import java.io.File; import java.util.ArrayList; +import java.util.Collection; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -55,7 +56,6 @@ import org.apache.pinot.segment.spi.loader.SegmentDirectoryLoaderRegistry; import org.apache.pinot.segment.spi.store.SegmentDirectory; import org.apache.pinot.spi.config.table.TableConfig; -import org.apache.pinot.spi.config.table.TableType; import org.apache.pinot.spi.data.Schema; import org.apache.pinot.spi.env.PinotConfiguration; import org.apache.pinot.spi.utils.CommonConstants.Broker; @@ -221,14 +221,14 @@ private BrokerResponseNative getBrokerResponse(PinotQuery pinotQuery, PlanMaker } // Broker side - Map dataTableMap = new HashMap<>(); + Map> dataTableMap = new HashMap<>(); try { // For multi-threaded BrokerReduceService, we cannot reuse the same data-table byte[] serializedResponse = instanceResponse.toDataTable().toBytes(); - dataTableMap.put(new ServerRoutingInstance("localhost", 1234, TableType.OFFLINE), - DataTableFactory.getDataTable(serializedResponse)); - dataTableMap.put(new ServerRoutingInstance("localhost", 1234, TableType.REALTIME), - DataTableFactory.getDataTable(serializedResponse)); + ServerRoutingInstance serverRoutingInstance = new ServerRoutingInstance("localhost", 1234); + dataTableMap.put(serverRoutingInstance, new ArrayList<>()); + dataTableMap.get(serverRoutingInstance).add(DataTableFactory.getDataTable(serializedResponse)); + dataTableMap.get(serverRoutingInstance).add(DataTableFactory.getDataTable(serializedResponse)); } catch (Exception e) { throw new RuntimeException(e); } @@ -245,7 +245,7 @@ private static List getSegmentContexts(List indexS } protected BrokerResponseNative reduceOnDataTable(BrokerRequest brokerRequest, BrokerRequest serverBrokerRequest, - Map dataTableMap) { + Map> dataTableMap) { BrokerReduceService brokerReduceService = new BrokerReduceService(new PinotConfiguration(Map.of(Broker.CONFIG_OF_MAX_REDUCE_THREADS_PER_QUERY, 2))); BrokerResponseNative brokerResponse = @@ -332,15 +332,15 @@ private BrokerResponseNative getBrokerResponseDistinctInstances(PinotQuery pinot } // Broker side - Map dataTableMap = new HashMap<>(); + Map> dataTableMap = new HashMap<>(); try { // For multi-threaded BrokerReduceService, we cannot reuse the same data-table byte[] serializedResponse1 = instanceResponse1.toDataTable().toBytes(); byte[] serializedResponse2 = instanceResponse2.toDataTable().toBytes(); - dataTableMap.put(new ServerRoutingInstance("localhost", 1234, TableType.OFFLINE), - DataTableFactory.getDataTable(serializedResponse1)); - dataTableMap.put(new ServerRoutingInstance("localhost", 1234, TableType.REALTIME), - DataTableFactory.getDataTable(serializedResponse2)); + ServerRoutingInstance serverRoutingInstance = new ServerRoutingInstance("localhost", 1234); + dataTableMap.put(serverRoutingInstance, new ArrayList<>()); + dataTableMap.get(serverRoutingInstance).add(DataTableFactory.getDataTable(serializedResponse1)); + dataTableMap.get(serverRoutingInstance).add(DataTableFactory.getDataTable(serializedResponse2)); } catch (Exception e) { throw new RuntimeException(e); } diff --git a/pinot-core/src/test/java/org/apache/pinot/queries/ExplainPlanQueriesTest.java b/pinot-core/src/test/java/org/apache/pinot/queries/ExplainPlanQueriesTest.java index 23fdd0c5451..858a94223f3 100644 --- a/pinot-core/src/test/java/org/apache/pinot/queries/ExplainPlanQueriesTest.java +++ b/pinot-core/src/test/java/org/apache/pinot/queries/ExplainPlanQueriesTest.java @@ -21,6 +21,7 @@ import java.io.File; import java.net.URL; import java.util.ArrayList; +import java.util.Collection; import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; @@ -389,15 +390,17 @@ private void checkWithQueryExecutor(String query, ResultTable expected, QueryExe // Use 2 Threads for 2 data-tables // Different segments are assigned to each set of DataTables. This is necessary to simulate scenarios where // certain segments may completely be pruned on one server but not the other. - Map dataTableMap = new HashMap<>(); + Map> dataTableMap = new HashMap<>(); try { + ServerRoutingInstance serverRoutingInstance = new ServerRoutingInstance("localhost", 1234); + // For multi-threaded BrokerReduceService, we cannot reuse the same data-table byte[] serializedResponse1 = instanceResponse1.toDataTable().toBytes(); - dataTableMap.put(new ServerRoutingInstance("localhost", 1234, TableType.OFFLINE), - DataTableFactory.getDataTable(serializedResponse1)); byte[] serializedResponse2 = instanceResponse2.toDataTable().toBytes(); - dataTableMap.put(new ServerRoutingInstance("localhost", 1234, TableType.REALTIME), - DataTableFactory.getDataTable(serializedResponse2)); + dataTableMap.computeIfAbsent(serverRoutingInstance, k -> new ArrayList<>()) + .add(DataTableFactory.getDataTable(serializedResponse1)); + dataTableMap.computeIfAbsent(serverRoutingInstance, k -> new ArrayList<>()) + .add(DataTableFactory.getDataTable(serializedResponse2)); } catch (Exception e) { throw new RuntimeException(e); } From 4e8656eb6b2fe74f9da78463c9b2de00909937b9 Mon Sep 17 00:00:00 2001 From: egalpin Date: Fri, 2 Aug 2024 17:26:46 -0700 Subject: [PATCH 02/49] Spotless linter --- .../requesthandler/BaseSingleStageBrokerRequestHandler.java | 2 +- .../requesthandler/SingleConnectionBrokerRequestHandler.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BaseSingleStageBrokerRequestHandler.java b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BaseSingleStageBrokerRequestHandler.java index a09a3f43990..1e389e583da 100644 --- a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BaseSingleStageBrokerRequestHandler.java +++ b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BaseSingleStageBrokerRequestHandler.java @@ -72,7 +72,6 @@ import org.apache.pinot.common.request.Identifier; import org.apache.pinot.common.request.Literal; import org.apache.pinot.common.request.PinotQuery; -import org.apache.pinot.core.transport.ServerQueryRoutingContext; import org.apache.pinot.common.response.BrokerResponse; import org.apache.pinot.common.response.ProcessingException; import org.apache.pinot.common.response.broker.BrokerResponseNative; @@ -87,6 +86,7 @@ import org.apache.pinot.core.routing.RoutingTable; import org.apache.pinot.core.routing.TimeBoundaryInfo; import org.apache.pinot.core.transport.ServerInstance; +import org.apache.pinot.core.transport.ServerQueryRoutingContext; import org.apache.pinot.core.util.GapfillUtils; import org.apache.pinot.query.parser.utils.ParserUtils; import org.apache.pinot.spi.auth.AuthorizationResult; diff --git a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/SingleConnectionBrokerRequestHandler.java b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/SingleConnectionBrokerRequestHandler.java index 74b69e0ab81..f4fbd693a14 100644 --- a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/SingleConnectionBrokerRequestHandler.java +++ b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/SingleConnectionBrokerRequestHandler.java @@ -39,7 +39,6 @@ import org.apache.pinot.common.metrics.BrokerMeter; import org.apache.pinot.common.metrics.BrokerQueryPhase; import org.apache.pinot.common.request.BrokerRequest; -import org.apache.pinot.core.transport.ServerQueryRoutingContext; import org.apache.pinot.common.response.broker.BrokerResponseNative; import org.apache.pinot.common.response.broker.QueryProcessingException; import org.apache.pinot.core.query.reduce.BrokerReduceService; @@ -47,6 +46,7 @@ import org.apache.pinot.core.transport.QueryResponse; import org.apache.pinot.core.transport.QueryRouter; import org.apache.pinot.core.transport.ServerInstance; +import org.apache.pinot.core.transport.ServerQueryRoutingContext; import org.apache.pinot.core.transport.ServerResponse; import org.apache.pinot.core.transport.ServerRoutingInstance; import org.apache.pinot.core.transport.server.routing.stats.ServerRoutingStatsManager; From 73748fd404b3c398dc3986490f809484cc304240 Mon Sep 17 00:00:00 2001 From: egalpin Date: Fri, 2 Aug 2024 17:37:50 -0700 Subject: [PATCH 03/49] Fixes test setups broken by method signature changes --- .../failuredetector/ConnectionFailureDetectorTest.java | 2 +- .../BaseSingleStageBrokerRequestHandlerTest.java | 7 +++---- 2 files changed, 4 insertions(+), 5 deletions(-) diff --git a/pinot-broker/src/test/java/org/apache/pinot/broker/failuredetector/ConnectionFailureDetectorTest.java b/pinot-broker/src/test/java/org/apache/pinot/broker/failuredetector/ConnectionFailureDetectorTest.java index 859b834f0ce..26627f01560 100644 --- a/pinot-broker/src/test/java/org/apache/pinot/broker/failuredetector/ConnectionFailureDetectorTest.java +++ b/pinot-broker/src/test/java/org/apache/pinot/broker/failuredetector/ConnectionFailureDetectorTest.java @@ -65,7 +65,7 @@ public void setUp() { @Test public void testConnectionFailure() { QueryResponse queryResponse = mock(QueryResponse.class); - when(queryResponse.getFailedServer()).thenReturn(new ServerRoutingInstance("localhost", 1234, TableType.OFFLINE)); + when(queryResponse.getFailedServer()).thenReturn(new ServerRoutingInstance("localhost", 1234)); // No failure detection when submitting the query _failureDetector.notifyQuerySubmitted(queryResponse); diff --git a/pinot-broker/src/test/java/org/apache/pinot/broker/requesthandler/BaseSingleStageBrokerRequestHandlerTest.java b/pinot-broker/src/test/java/org/apache/pinot/broker/requesthandler/BaseSingleStageBrokerRequestHandlerTest.java index f1a6dfe33f1..78a4d2a058c 100644 --- a/pinot-broker/src/test/java/org/apache/pinot/broker/requesthandler/BaseSingleStageBrokerRequestHandlerTest.java +++ b/pinot-broker/src/test/java/org/apache/pinot/broker/requesthandler/BaseSingleStageBrokerRequestHandlerTest.java @@ -38,6 +38,7 @@ import org.apache.pinot.common.response.broker.BrokerResponseNative; import org.apache.pinot.core.routing.RoutingTable; import org.apache.pinot.core.transport.ServerInstance; +import org.apache.pinot.core.transport.ServerQueryRoutingContext; import org.apache.pinot.spi.config.table.TableConfig; import org.apache.pinot.spi.config.table.TenantConfig; import org.apache.pinot.spi.env.PinotConfiguration; @@ -191,10 +192,8 @@ public void shutDown() { @Override protected BrokerResponseNative processBrokerRequest(long requestId, BrokerRequest originalBrokerRequest, - BrokerRequest serverBrokerRequest, @Nullable BrokerRequest offlineBrokerRequest, - @Nullable Map, List>> offlineRoutingTable, - @Nullable BrokerRequest realtimeBrokerRequest, - @Nullable Map, List>> realtimeRoutingTable, long timeoutMs, + BrokerRequest serverBrokerRequest, + @Nullable Map> queryRoutingTable, long timeoutMs, ServerStats serverStats, RequestContext requestContext) throws Exception { testRequestId[0] = requestId; From c8285ec2dd9b11cdda4c598c7e03bbfa5f5babde Mon Sep 17 00:00:00 2001 From: egalpin Date: Tue, 6 Aug 2024 15:20:12 -0700 Subject: [PATCH 04/49] Moves placement of appending tableName metadata --- .../pinot/core/plan/GlobalPlanImplV0.java | 4 +++ .../executor/ServerQueryExecutorV1Impl.java | 30 ++++++++----------- 2 files changed, 16 insertions(+), 18 deletions(-) diff --git a/pinot-core/src/main/java/org/apache/pinot/core/plan/GlobalPlanImplV0.java b/pinot-core/src/main/java/org/apache/pinot/core/plan/GlobalPlanImplV0.java index 94cccc45ae9..073b1cae68b 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/plan/GlobalPlanImplV0.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/plan/GlobalPlanImplV0.java @@ -19,6 +19,7 @@ package org.apache.pinot.core.plan; import java.util.concurrent.TimeoutException; +import org.apache.pinot.common.datatable.DataTable; import org.apache.pinot.core.operator.InstanceResponseOperator; import org.apache.pinot.core.operator.blocks.InstanceResponseBlock; import org.slf4j.Logger; @@ -47,6 +48,7 @@ public PlanNode getPlanNode() { @Override public InstanceResponseBlock execute() throws TimeoutException { + // TODO(egalpin) add code to add the table name meta data here long startTime = System.currentTimeMillis(); InstanceResponseOperator instanceResponseOperator = _instanceResponsePlanNode.run(); long endTime1 = System.currentTimeMillis(); @@ -57,6 +59,8 @@ public InstanceResponseBlock execute() InstanceResponseBlock instanceResponseBlock = instanceResponseOperator.nextBlock(); long endTime2 = System.currentTimeMillis(); LOGGER.debug("InstanceResponseOperator.nextBlock() took: {}ms", endTime2 - endTime1); + instanceResponseBlock.addMetadata(DataTable.MetadataKey.TABLE.getName(), + _instanceResponsePlanNode._queryContext.getTableName()); return instanceResponseBlock; } } diff --git a/pinot-core/src/main/java/org/apache/pinot/core/query/executor/ServerQueryExecutorV1Impl.java b/pinot-core/src/main/java/org/apache/pinot/core/query/executor/ServerQueryExecutorV1Impl.java index c011e5074bc..8c2906db541 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/query/executor/ServerQueryExecutorV1Impl.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/query/executor/ServerQueryExecutorV1Impl.java @@ -133,26 +133,20 @@ public synchronized void shutDown() { @Override public InstanceResponseBlock execute(ServerQueryRequest queryRequest, ExecutorService executorService, @Nullable ResultsBlockStreamer streamer) { - InstanceResponseBlock responseBlock; - if (!queryRequest.isEnableTrace()) { - responseBlock = executeInternal(queryRequest, executorService, streamer); - } else { - try { - 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); - responseBlock = executeInternal(queryRequest, executorService, streamer); - } finally { - Tracing.getTracer().unregister(); - } + return executeInternal(queryRequest, executorService, streamer); + } + try { + 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, streamer); + } finally { + Tracing.getTracer().unregister(); } - - responseBlock.addMetadata(MetadataKey.TABLE.getName(), queryRequest.getTableNameWithType()); - return responseBlock; } private InstanceResponseBlock executeInternal(ServerQueryRequest queryRequest, ExecutorService executorService, From 35188eae0f1f45320c8d57a944c6fcd72b5e3809 Mon Sep 17 00:00:00 2001 From: egalpin Date: Tue, 6 Aug 2024 15:24:26 -0700 Subject: [PATCH 05/49] Removes unused import --- .../broker/failuredetector/ConnectionFailureDetectorTest.java | 1 - 1 file changed, 1 deletion(-) diff --git a/pinot-broker/src/test/java/org/apache/pinot/broker/failuredetector/ConnectionFailureDetectorTest.java b/pinot-broker/src/test/java/org/apache/pinot/broker/failuredetector/ConnectionFailureDetectorTest.java index 26627f01560..bd8e87fd062 100644 --- a/pinot-broker/src/test/java/org/apache/pinot/broker/failuredetector/ConnectionFailureDetectorTest.java +++ b/pinot-broker/src/test/java/org/apache/pinot/broker/failuredetector/ConnectionFailureDetectorTest.java @@ -26,7 +26,6 @@ import org.apache.pinot.common.metrics.MetricValueUtils; import org.apache.pinot.core.transport.QueryResponse; import org.apache.pinot.core.transport.ServerRoutingInstance; -import org.apache.pinot.spi.config.table.TableType; import org.apache.pinot.spi.env.PinotConfiguration; import org.apache.pinot.spi.metrics.PinotMetricUtils; import org.apache.pinot.spi.utils.CommonConstants.Broker; From 2ef90476805ac9e1b3bd4e5b5e47b44eee8c0e8d Mon Sep 17 00:00:00 2001 From: egalpin Date: Tue, 6 Aug 2024 15:24:38 -0700 Subject: [PATCH 06/49] Updates signatures to match --- .../tests/OfflineGRPCServerIntegrationTest.java | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/OfflineGRPCServerIntegrationTest.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/OfflineGRPCServerIntegrationTest.java index 6408fd8f31d..6815e71f6ad 100644 --- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/OfflineGRPCServerIntegrationTest.java +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/OfflineGRPCServerIntegrationTest.java @@ -20,6 +20,8 @@ import java.io.File; import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; import java.util.HashMap; import java.util.Iterator; import java.util.List; @@ -181,7 +183,7 @@ private DataTable collectNonStreamingRequestResult(Iterator streamingResponses, DataTable nonStreamResultDataTable) throws Exception { - Map dataTableMap = new HashMap<>(); + Map> dataTableMap = new HashMap<>(); DataSchema cachedDataSchema = null; while (streamingResponses.hasNext()) { Server.ServerResponse streamingResponse = streamingResponses.next(); @@ -196,7 +198,7 @@ private void collectAndCompareResult(String sql, Iterator assertNotNull(dataTable.getDataSchema()); cachedDataSchema = dataTable.getDataSchema(); // adding them to a fake dataTableMap for reduce - dataTableMap.put(mock(ServerRoutingInstance.class), dataTable); + dataTableMap.put(mock(ServerRoutingInstance.class), Collections.singleton(dataTable)); } else { // compare result dataTable against nonStreamingResultDataTable // Process server response. @@ -207,8 +209,8 @@ private void collectAndCompareResult(String sql, Iterator DATATABLE_REDUCER_CONTEXT, mock(BrokerMetrics.class)); BrokerResponseNative nonStreamBrokerResponse = new BrokerResponseNative(); reducer.reduceAndSetResults("mytable_OFFLINE", nonStreamResultDataTable.getDataSchema(), - Map.of(mock(ServerRoutingInstance.class), nonStreamResultDataTable), nonStreamBrokerResponse, - DATATABLE_REDUCER_CONTEXT, mock(BrokerMetrics.class)); + Map.of(mock(ServerRoutingInstance.class), Collections.singleton(nonStreamResultDataTable)), + nonStreamBrokerResponse, DATATABLE_REDUCER_CONTEXT, mock(BrokerMetrics.class)); assertEquals(streamingBrokerResponse.getResultTable().getRows().size(), nonStreamBrokerResponse.getResultTable().getRows().size()); From a9eaa1fe6cc8cc5954dda6c7b60b5087020266f8 Mon Sep 17 00:00:00 2001 From: egalpin Date: Tue, 6 Aug 2024 16:28:28 -0700 Subject: [PATCH 07/49] Makes QueryResponse return iterable of ServerResponses without nested Map --- .../SingleConnectionBrokerRequestHandler.java | 7 ++-- .../core/transport/AsyncQueryResponse.java | 29 +++++++++++----- .../pinot/core/transport/QueryResponse.java | 5 +-- .../core/transport/QueryRoutingTest.java | 33 +++++++++---------- 4 files changed, 43 insertions(+), 31 deletions(-) diff --git a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/SingleConnectionBrokerRequestHandler.java b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/SingleConnectionBrokerRequestHandler.java index f4fbd693a14..6d6f1da983c 100644 --- a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/SingleConnectionBrokerRequestHandler.java +++ b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/SingleConnectionBrokerRequestHandler.java @@ -112,7 +112,7 @@ protected BrokerResponseNative processBrokerRequest(long requestId, BrokerReques AsyncQueryResponse asyncQueryResponse = _queryRouter.submitQuery(requestId, rawTableName, queryRoutingTable, timeoutMs); _failureDetector.notifyQuerySubmitted(asyncQueryResponse); - Map> finalResponses = asyncQueryResponse.getFinalResponses(); + Map> finalResponses = asyncQueryResponse.getFinalResponses(); if (asyncQueryResponse.getStatus() == QueryResponse.Status.TIMED_OUT) { _brokerMetrics.addMeteredTableValue(rawTableName, BrokerMeter.BROKER_RESPONSES_WITH_TIMEOUTS, 1); } @@ -126,9 +126,8 @@ protected BrokerResponseNative processBrokerRequest(long requestId, BrokerReques long totalResponseSize = 0; Map> dataTableMap = Maps.newHashMapWithExpectedSize(numServersQueried); List serversNotResponded = new ArrayList<>(); - for (Map.Entry> serverResponses : finalResponses.entrySet()) { - for (Map.Entry responsePair : serverResponses.getValue().entrySet()) { - ServerResponse response = responsePair.getValue(); + for (Map.Entry> serverResponses : finalResponses.entrySet()) { + for (ServerResponse response : serverResponses.getValue()) { DataTable dataTable = response.getDataTable(); if (dataTable != null) { dataTableMap.computeIfAbsent(serverResponses.getKey(), k -> new ArrayList<>()).add(dataTable); diff --git a/pinot-core/src/main/java/org/apache/pinot/core/transport/AsyncQueryResponse.java b/pinot-core/src/main/java/org/apache/pinot/core/transport/AsyncQueryResponse.java index 000207752bd..8d94a6aadcc 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/transport/AsyncQueryResponse.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/transport/AsyncQueryResponse.java @@ -18,6 +18,7 @@ */ package org.apache.pinot.core.transport; +import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -90,25 +91,37 @@ public int getNumServersResponded() { return _numServersResponded.get(); } + private Map> getFlatResponses() { + Map> flattened = new HashMap<>(); + for (Map.Entry> serverResponses : _responses.entrySet()) { + ServerRoutingInstance serverRoutingInstance = serverResponses.getKey(); + + for (ServerResponse serverResponse : serverResponses.getValue().values()) { + flattened.computeIfAbsent(serverRoutingInstance, k -> new ArrayList<>()).add(serverResponse); + } + } + + return flattened; + } + @Override - public Map> getCurrentResponses() { - return _responses; + public Map> getCurrentResponses() { + return getFlatResponses(); } @Override - public Map> getFinalResponses() + public Map> getFinalResponses() throws InterruptedException { + Map> flatResponses = getFlatResponses(); try { boolean finish = _countDownLatch.await(_maxEndTimeMs - System.currentTimeMillis(), TimeUnit.MILLISECONDS); _status.compareAndSet(Status.IN_PROGRESS, finish ? Status.COMPLETED : Status.TIMED_OUT); - return _responses; + return flatResponses; } finally { // Update ServerRoutingStats for query completion. This is done here to ensure that the stats are updated for // servers even if the query times out or if servers have not responded. - for (Map.Entry> serverResponses : _responses.entrySet()) { - for (Map.Entry responsePair : serverResponses.getValue().entrySet()) { - ServerResponse response = responsePair.getValue(); - + for (Map.Entry> serverResponses : flatResponses.entrySet()) { + for (ServerResponse response : serverResponses.getValue()) { // ServerResponse returns -1 if responseDelayMs is not set. This indicates that a response was not received // from the server. Hence we set the latency to the timeout value. long latency = diff --git a/pinot-core/src/main/java/org/apache/pinot/core/transport/QueryResponse.java b/pinot-core/src/main/java/org/apache/pinot/core/transport/QueryResponse.java index 14cd932ea48..caa91a3ff75 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/transport/QueryResponse.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/transport/QueryResponse.java @@ -18,6 +18,7 @@ */ package org.apache.pinot.core.transport; +import java.util.List; import java.util.Map; import javax.annotation.Nullable; import javax.annotation.concurrent.ThreadSafe; @@ -49,12 +50,12 @@ enum Status { /** * Returns the current server responses without blocking. */ - Map> getCurrentResponses(); + Map> getCurrentResponses(); /** * Waits until the query is done (COMPLETED, FAILED or TIMED_OUT) and returns the final server responses. */ - Map> getFinalResponses() + Map> getFinalResponses() throws InterruptedException; /** diff --git a/pinot-core/src/test/java/org/apache/pinot/core/transport/QueryRoutingTest.java b/pinot-core/src/test/java/org/apache/pinot/core/transport/QueryRoutingTest.java index d0b3179e4a6..e1490bb97a1 100644 --- a/pinot-core/src/test/java/org/apache/pinot/core/transport/QueryRoutingTest.java +++ b/pinot-core/src/test/java/org/apache/pinot/core/transport/QueryRoutingTest.java @@ -20,7 +20,6 @@ import com.google.common.util.concurrent.Futures; import java.io.IOException; -import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -173,11 +172,11 @@ public void testValidResponse() // OFFLINE only AsyncQueryResponse asyncQueryResponse = _queryRouter.submitQuery(requestId, "testTable", OFFLINE_ROUTING_TABLE, 600_000L); - Map> response = asyncQueryResponse.getFinalResponses(); + Map> response = asyncQueryResponse.getFinalResponses(); assertEquals(response.size(), 1); assertTrue(response.containsKey(SERVER_ROUTING_INSTANCE)); assertEquals(response.get(SERVER_ROUTING_INSTANCE).size(), 1); - ServerResponse serverResponse = new ArrayList<>(response.get(SERVER_ROUTING_INSTANCE).values()).get(0); + ServerResponse serverResponse = response.get(SERVER_ROUTING_INSTANCE).get(0); assertNotNull(serverResponse.getDataTable()); assertEquals(serverResponse.getResponseSize(), offlineResponseBytes.length); // 2 requests - query submit and query response. @@ -191,7 +190,7 @@ public void testValidResponse() assertEquals(response.size(), 1); assertTrue(response.containsKey(SERVER_ROUTING_INSTANCE)); assertEquals(response.get(SERVER_ROUTING_INSTANCE).size(), 1); - serverResponse = new ArrayList<>(response.get(SERVER_ROUTING_INSTANCE).values()).get(0); + serverResponse = response.get(SERVER_ROUTING_INSTANCE).get(0); assertNotNull(serverResponse.getDataTable()); assertEquals(serverResponse.getResponseSize(), realtimeResponseBytes.length); _requestCount += 2; @@ -206,7 +205,7 @@ public void testValidResponse() assertEquals(response.get(SERVER_ROUTING_INSTANCE).size(), 2); int accountedFor = 0; - for (ServerResponse serverResponse1 : response.get(SERVER_ROUTING_INSTANCE).values()) { + for (ServerResponse serverResponse1 : response.get(SERVER_ROUTING_INSTANCE)) { assertNotNull(serverResponse1.getDataTable()); if (serverResponse1.getDataTable().getMetadata().get(MetadataKey.QUERY_HASH.getName()) .equals(offlineDataTable.getMetadata().get(MetadataKey.QUERY_HASH.getName()))) { @@ -240,11 +239,11 @@ public void testInvalidResponse() long startTimeMs = System.currentTimeMillis(); AsyncQueryResponse asyncQueryResponse = _queryRouter.submitQuery(requestId, "testTable", OFFLINE_ROUTING_TABLE, 1_000L); - Map> response = asyncQueryResponse.getFinalResponses(); + Map> response = asyncQueryResponse.getFinalResponses(); assertEquals(response.size(), 1); assertTrue(response.containsKey(SERVER_ROUTING_INSTANCE)); assertEquals(response.get(SERVER_ROUTING_INSTANCE).size(), 1); - ServerResponse serverResponse = new ArrayList<>(response.get(SERVER_ROUTING_INSTANCE).values()).get(0); + ServerResponse serverResponse = response.get(SERVER_ROUTING_INSTANCE).get(0); assertNull(serverResponse.getDataTable()); assertEquals(serverResponse.getResponseDelayMs(), -1); assertEquals(serverResponse.getResponseSize(), 0); @@ -281,11 +280,11 @@ public void testNonMatchingRequestId() long startTimeMs = System.currentTimeMillis(); AsyncQueryResponse asyncQueryResponse = _queryRouter.submitQuery(requestId + 1, "testTable", OFFLINE_ROUTING_TABLE, 1_000L); - Map> response = asyncQueryResponse.getFinalResponses(); + Map> response = asyncQueryResponse.getFinalResponses(); assertEquals(response.size(), 1); assertTrue(response.containsKey(SERVER_ROUTING_INSTANCE)); assertEquals(response.get(SERVER_ROUTING_INSTANCE).size(), 1); - ServerResponse serverResponse = new ArrayList<>(response.get(SERVER_ROUTING_INSTANCE).values()).get(0); + ServerResponse serverResponse = response.get(SERVER_ROUTING_INSTANCE).get(0); assertNull(serverResponse.getDataTable()); assertEquals(serverResponse.getResponseDelayMs(), -1); assertEquals(serverResponse.getResponseSize(), 0); @@ -330,11 +329,11 @@ public void testServerDown() // Shut down the server before getting the response queryServer.shutDown(); - Map> response = asyncQueryResponse.getFinalResponses(); + Map> response = asyncQueryResponse.getFinalResponses(); assertEquals(response.size(), 1); assertTrue(response.containsKey(SERVER_ROUTING_INSTANCE)); assertEquals(response.get(SERVER_ROUTING_INSTANCE).size(), 1); - ServerResponse serverResponse = new ArrayList<>(response.get(SERVER_ROUTING_INSTANCE).values()).get(0); + ServerResponse serverResponse = response.get(SERVER_ROUTING_INSTANCE).get(0); assertNull(serverResponse.getDataTable()); assertEquals(serverResponse.getResponseDelayMs(), -1); assertEquals(serverResponse.getResponseSize(), 0); @@ -352,7 +351,7 @@ public void testServerDown() assertEquals(response.size(), 1); assertTrue(response.containsKey(SERVER_ROUTING_INSTANCE)); assertEquals(response.get(SERVER_ROUTING_INSTANCE).size(), 1); - serverResponse = new ArrayList<>(response.get(SERVER_ROUTING_INSTANCE).values()).get(0); + serverResponse = response.get(SERVER_ROUTING_INSTANCE).get(0); assertNull(serverResponse.getDataTable()); assertEquals(serverResponse.getSubmitDelayMs(), -1); assertEquals(serverResponse.getResponseDelayMs(), -1); @@ -413,13 +412,13 @@ public void testSkipUnavailableServer() // Submit the query with skipUnavailableServers=true, the single started server should return a valid response long startTime = System.currentTimeMillis(); AsyncQueryResponse asyncQueryResponse = _queryRouter.submitQuery(requestId, "testTable", routingTable, 10_000L); - Map> response = asyncQueryResponse.getFinalResponses(); + Map> response = asyncQueryResponse.getFinalResponses(); assertEquals(response.size(), 2); assertTrue(response.containsKey(serverRoutingInstance1)); assertTrue(response.containsKey(serverRoutingInstance2)); - ServerResponse serverResponse1 = new ArrayList<>(response.get(serverRoutingInstance1).values()).get(0); - ServerResponse serverResponse2 = new ArrayList<>(response.get(serverRoutingInstance2).values()).get(0); + ServerResponse serverResponse1 = response.get(serverRoutingInstance1).get(0); + ServerResponse serverResponse2 = response.get(serverRoutingInstance2).get(0); assertNotNull(serverResponse1.getDataTable()); assertNull(serverResponse2.getDataTable()); assertTrue(serverResponse1.getResponseDelayMs() > 500); // > response delay set by getQueryServer @@ -458,8 +457,8 @@ public void testSkipUnavailableServer() assertTrue(response.containsKey(serverRoutingInstance1)); assertTrue(response.containsKey(serverRoutingInstance2)); - serverResponse1 = new ArrayList<>(response.get(serverRoutingInstance1).values()).get(0); - serverResponse2 = new ArrayList<>(response.get(serverRoutingInstance2).values()).get(0); + serverResponse1 = response.get(serverRoutingInstance1).get(0); + serverResponse2 = response.get(serverRoutingInstance2).get(0); assertNull(serverResponse1.getDataTable()); assertNull(serverResponse2.getDataTable()); assertTrue(serverResponse1.getResponseDelayMs() < 100); From 09d9e201a219a01831fa642614bb5353de82916e Mon Sep 17 00:00:00 2001 From: egalpin Date: Tue, 6 Aug 2024 16:46:49 -0700 Subject: [PATCH 08/49] Updates scala to match iterable server responses --- .../reader/PinotServerDataFetcher.scala | 57 ++++++++++--------- 1 file changed, 30 insertions(+), 27 deletions(-) diff --git a/pinot-connectors/pinot-spark-common/src/main/scala/org/apache/pinot/connector/spark/common/reader/PinotServerDataFetcher.scala b/pinot-connectors/pinot-spark-common/src/main/scala/org/apache/pinot/connector/spark/common/reader/PinotServerDataFetcher.scala index 115b8ec587f..ee6db3081a4 100644 --- a/pinot-connectors/pinot-spark-common/src/main/scala/org/apache/pinot/connector/spark/common/reader/PinotServerDataFetcher.scala +++ b/pinot-connectors/pinot-spark-common/src/main/scala/org/apache/pinot/connector/spark/common/reader/PinotServerDataFetcher.scala @@ -7,7 +7,7 @@ * "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 + * 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 @@ -40,9 +40,9 @@ import scala.collection.JavaConverters._ * Eg: offline-server1: segment1, segment2, segment3 */ private[reader] class PinotServerDataFetcher( - partitionId: Int, - pinotSplit: PinotSplit, - dataSourceOptions: PinotDataSourceReadOptions) + partitionId: Int, + pinotSplit: PinotSplit, + dataSourceOptions: PinotDataSourceReadOptions) extends Logging { private val brokerId = "apache_spark" private val metricsRegistry = PinotMetricUtils.getPinotMetricsRegistry @@ -53,33 +53,32 @@ private[reader] class PinotServerDataFetcher( // TODO add support for TLS-secured server def fetchData(): List[DataTable] = { - val routingTableForRequest = createRoutingTableForRequest() - val requestStartTime = System.nanoTime() - val pinotServerAsyncQueryResponse = pinotSplit.serverAndSegments.serverType match { + val routingTableForRequest = + pinotSplit.serverAndSegments.serverType match { case TableType.REALTIME => - val realtimeBrokerRequest = - CalciteSqlCompiler.compileToBrokerRequest(pinotSplit.query.realtimeSelectQuery) - submitRequestToPinotServer(null, null, realtimeBrokerRequest, routingTableForRequest) + val realtimeBrokerRequest = CalciteSqlCompiler.compileToBrokerRequest(pinotSplit.query.realtimeSelectQuery) + createRoutingTableForRequest(realtimeBrokerRequest) + case TableType.OFFLINE => - val offlineBrokerRequest = - CalciteSqlCompiler.compileToBrokerRequest(pinotSplit.query.offlineSelectQuery) - submitRequestToPinotServer(offlineBrokerRequest, routingTableForRequest, null, null) + val offlineBrokerRequest = CalciteSqlCompiler.compileToBrokerRequest(pinotSplit.query.offlineSelectQuery) + createRoutingTableForRequest(offlineBrokerRequest) } - val pinotServerResponse = pinotServerAsyncQueryResponse.getFinalResponses.values().asScala.toList + val pinotServerAsyncQueryResponse = submitRequestToPinotServer(routingTableForRequest) + val pinotServerResponse = pinotServerAsyncQueryResponse.getFinalResponses.values().asScala.flatMap(_.asScala.toList).toList logInfo(s"Pinot server total response time in millis: ${System.nanoTime() - requestStartTime}") closePinotServerConnection() pinotServerResponse.foreach { response => - logInfo( - s"Request stats; " + - s"responseSize: ${response.getResponseSize}, " + - s"responseDelayMs: ${response.getResponseDelayMs}, " + - s"deserializationTimeMs: ${response.getDeserializationTimeMs}, " + - s"submitDelayMs: ${response.getSubmitDelayMs}" - ) + logInfo( + s"Request stats; " + + s"responseSize: ${response.getResponseSize}, " + + s"responseDelayMs: ${response.getResponseDelayMs}, " + + s"deserializationTimeMs: ${response.getDeserializationTimeMs}, " + + s"submitDelayMs: ${response.getSubmitDelayMs}" + ) } val dataTables = pinotServerResponse @@ -93,20 +92,24 @@ private[reader] class PinotServerDataFetcher( dataTables.filter(_.getNumberOfRows > 0) } - private def createRoutingTableForRequest(): JMap[ServerInstance, Pair[JList[String], JList[String]]] = { + // TODO(egalpin): update to use ServerQueryRoutingContext + private def createRoutingTableForRequest(brokerRequest: BrokerRequest): JMap[ServerInstance, JList[ServerQueryRoutingContext]] = { val nullZkId: String = null val instanceConfig = new InstanceConfig(nullZkId) instanceConfig.setHostName(pinotSplit.serverAndSegments.serverHost) instanceConfig.setPort(pinotSplit.serverAndSegments.serverPort) // TODO: support netty-sec val serverInstance = new ServerInstance(instanceConfig) + val serverQueryRoutingContext = new ServerQueryRoutingContext(brokerRequest, + Pair.of(pinotSplit.serverAndSegments.segments.asJava, List[String]().asJava), + serverInstance.toServerRoutingInstance(serverInstance.isTlsEnabled)) Map( - serverInstance -> Pair.of(pinotSplit.serverAndSegments.segments.asJava, List[String]().asJava) + serverInstance -> List(serverQueryRoutingContext).asJava ).asJava } private def submitRequestToPinotServer( - queryRoutingTable JMap[ServerInstance, JList[ServerQueryRoutingContext]]): AsyncQueryResponse = { + queryRoutingTable: JMap[ServerInstance, JList[ServerQueryRoutingContext]]): AsyncQueryResponse = { logInfo(s"Sending request to ${pinotSplit.serverAndSegments.toString}") queryRouter.submitQuery( partitionId, @@ -125,9 +128,9 @@ private[reader] class PinotServerDataFetcher( object PinotServerDataFetcher { def apply( - partitionId: Int, - pinotSplit: PinotSplit, - dataSourceOptions: PinotDataSourceReadOptions): PinotServerDataFetcher = { + partitionId: Int, + pinotSplit: PinotSplit, + dataSourceOptions: PinotDataSourceReadOptions): PinotServerDataFetcher = { new PinotServerDataFetcher(partitionId, pinotSplit, dataSourceOptions) } } From a692f624d2c0673b94418e068a052867fbbd32d9 Mon Sep 17 00:00:00 2001 From: egalpin Date: Tue, 6 Aug 2024 17:18:28 -0700 Subject: [PATCH 09/49] Updates license header to comply with linter --- .../connector/spark/common/reader/PinotServerDataFetcher.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pinot-connectors/pinot-spark-common/src/main/scala/org/apache/pinot/connector/spark/common/reader/PinotServerDataFetcher.scala b/pinot-connectors/pinot-spark-common/src/main/scala/org/apache/pinot/connector/spark/common/reader/PinotServerDataFetcher.scala index ee6db3081a4..4bae4c7dfb2 100644 --- a/pinot-connectors/pinot-spark-common/src/main/scala/org/apache/pinot/connector/spark/common/reader/PinotServerDataFetcher.scala +++ b/pinot-connectors/pinot-spark-common/src/main/scala/org/apache/pinot/connector/spark/common/reader/PinotServerDataFetcher.scala @@ -7,7 +7,7 @@ * "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 + * 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 From 5359950163a82461ef841a59bfabb7b1a657020f Mon Sep 17 00:00:00 2001 From: egalpin Date: Wed, 7 Aug 2024 13:22:30 -0700 Subject: [PATCH 10/49] Makes query routing table non-nullable --- .../BaseSingleStageBrokerRequestHandler.java | 14 ++++++++------ .../requesthandler/GrpcBrokerRequestHandler.java | 5 ++--- .../SingleConnectionBrokerRequestHandler.java | 5 ++--- .../BaseSingleStageBrokerRequestHandlerTest.java | 6 +++--- 4 files changed, 15 insertions(+), 15 deletions(-) diff --git a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BaseSingleStageBrokerRequestHandler.java b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BaseSingleStageBrokerRequestHandler.java index 1e389e583da..c4743cbe96f 100644 --- a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BaseSingleStageBrokerRequestHandler.java +++ b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BaseSingleStageBrokerRequestHandler.java @@ -25,6 +25,7 @@ import java.net.URI; import java.util.ArrayList; import java.util.Collections; +import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; @@ -621,7 +622,7 @@ protected BrokerResponse handleRequest(long requestId, String query, @Nullable S // Calculate routing table for the query // TODO: Modify RoutingManager interface to directly take PinotQuery long routingStartTimeNs = System.nanoTime(); - Map> queryRoutingTable = null; + Map> queryRoutingTable = new HashMap<>(); List unavailableSegments = new ArrayList<>(); int numPrunedSegmentsTotal = 0; @@ -635,7 +636,9 @@ protected BrokerResponse handleRequest(long requestId, String query, @Nullable S } } - if (realtimeBrokerRequest != null) { + // TODO: Assess if the Explain Plan Query should also be routed to REALTIME servers for HYBRID tables + if (realtimeBrokerRequest != null && (!pinotQuery.isExplain() || offlineBrokerRequest != null)) { + // Don't send explain queries to realtime for OFFLINE or HYBRID tables Integer numPrunedSegments = updateRoutingTable(requestId, realtimeBrokerRequest, queryRoutingTable, unavailableSegments); if (numPrunedSegments == null) { @@ -741,15 +744,14 @@ protected BrokerResponse handleRequest(long requestId, String query, @Nullable S // - Compile time function invocation // - Literal only queries // - Any rewrites - if (pinotQuery.isExplain()) { +// if (pinotQuery.isExplain()) { // Update routing tables to only send request to offline servers for OFFLINE and HYBRID tables. - // TODO: Assess if the Explain Plan Query should also be routed to REALTIME servers for HYBRID tables // if (offlineRoutingTable != null) { // // For OFFLINE and HYBRID tables, don't send EXPLAIN query to realtime servers. // realtimeBrokerRequest = null; // realtimeRoutingTable = null; // } - } +// } BrokerResponseNative brokerResponse; if (_queriesById != null) { // Start to track the running query for cancellation just before sending it out to servers to avoid any @@ -1882,7 +1884,7 @@ private static void attachTimeBoundary(PinotQuery pinotQuery, TimeBoundaryInfo t */ protected abstract BrokerResponseNative processBrokerRequest(long requestId, BrokerRequest originalBrokerRequest, BrokerRequest serverBrokerRequest, - @Nullable Map> queryRoutingTable, long timeoutMs, + Map> queryRoutingTable, long timeoutMs, ServerStats serverStats, RequestContext requestContext) throws Exception; diff --git a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/GrpcBrokerRequestHandler.java b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/GrpcBrokerRequestHandler.java index 6b716412983..d1dd8a30e6d 100644 --- a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/GrpcBrokerRequestHandler.java +++ b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/GrpcBrokerRequestHandler.java @@ -24,7 +24,6 @@ import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.TimeUnit; -import javax.annotation.Nullable; import javax.annotation.concurrent.ThreadSafe; import org.apache.pinot.broker.broker.AccessControlFactory; import org.apache.pinot.broker.queryquota.QueryQuotaManager; @@ -75,12 +74,12 @@ public void shutDown() { @Override protected BrokerResponseNative processBrokerRequest(long requestId, BrokerRequest originalBrokerRequest, BrokerRequest serverBrokerRequest, - @Nullable Map> queryRoutingTable, long timeoutMs, + Map> queryRoutingTable, long timeoutMs, ServerStats serverStats, RequestContext requestContext) throws Exception { // TODO: Support failure detection // TODO: Add servers queried/responded stats - assert queryRoutingTable != null && !queryRoutingTable.isEmpty(); + assert !queryRoutingTable.isEmpty(); Map> responseMap = new HashMap<>(); sendRequest(requestId, queryRoutingTable, responseMap, requestContext.isSampledRequest()); long reduceStartTimeNs = System.nanoTime(); diff --git a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/SingleConnectionBrokerRequestHandler.java b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/SingleConnectionBrokerRequestHandler.java index 6d6f1da983c..6ed7d618e10 100644 --- a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/SingleConnectionBrokerRequestHandler.java +++ b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/SingleConnectionBrokerRequestHandler.java @@ -24,7 +24,6 @@ import java.util.List; import java.util.Map; import java.util.concurrent.TimeUnit; -import javax.annotation.Nullable; import javax.annotation.concurrent.ThreadSafe; import org.apache.pinot.broker.broker.AccessControlFactory; import org.apache.pinot.broker.failuredetector.FailureDetector; @@ -99,10 +98,10 @@ public void shutDown() { @Override protected BrokerResponseNative processBrokerRequest(long requestId, BrokerRequest originalBrokerRequest, BrokerRequest serverBrokerRequest, - @Nullable Map> queryRoutingTable, long timeoutMs, + Map> queryRoutingTable, long timeoutMs, ServerStats serverStats, RequestContext requestContext) throws Exception { - assert queryRoutingTable != null && !queryRoutingTable.isEmpty(); + assert !queryRoutingTable.isEmpty(); if (requestContext.isSampledRequest()) { serverBrokerRequest.getPinotQuery().putToQueryOptions(CommonConstants.Broker.Request.TRACE, "true"); } diff --git a/pinot-broker/src/test/java/org/apache/pinot/broker/requesthandler/BaseSingleStageBrokerRequestHandlerTest.java b/pinot-broker/src/test/java/org/apache/pinot/broker/requesthandler/BaseSingleStageBrokerRequestHandlerTest.java index 78a4d2a058c..dd7a64034e5 100644 --- a/pinot-broker/src/test/java/org/apache/pinot/broker/requesthandler/BaseSingleStageBrokerRequestHandlerTest.java +++ b/pinot-broker/src/test/java/org/apache/pinot/broker/requesthandler/BaseSingleStageBrokerRequestHandlerTest.java @@ -24,7 +24,6 @@ import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; -import javax.annotation.Nullable; import org.apache.commons.lang3.tuple.Pair; import org.apache.helix.model.InstanceConfig; import org.apache.pinot.broker.broker.AllowAllAccessControlFactory; @@ -155,7 +154,8 @@ public void testGetActualColumnNameCaseInSensitive() { } @Test - public void testCancelQuery() { + public void testCancelQuery() + throws InterruptedException { String tableName = "myTable_OFFLINE"; // Mock pretty much everything until the query can be submitted. TableCache tableCache = mock(TableCache.class); @@ -193,7 +193,7 @@ public void shutDown() { @Override protected BrokerResponseNative processBrokerRequest(long requestId, BrokerRequest originalBrokerRequest, BrokerRequest serverBrokerRequest, - @Nullable Map> queryRoutingTable, long timeoutMs, + Map> queryRoutingTable, long timeoutMs, ServerStats serverStats, RequestContext requestContext) throws Exception { testRequestId[0] = requestId; From f441389611d61dd5d2a33932d6a9178bb417b17d Mon Sep 17 00:00:00 2001 From: egalpin Date: Wed, 7 Aug 2024 16:02:18 -0700 Subject: [PATCH 11/49] Moves table name metadata setting to server-side --- .../java/org/apache/pinot/core/plan/GlobalPlanImplV0.java | 4 ---- .../org/apache/pinot/core/query/scheduler/QueryScheduler.java | 1 + 2 files changed, 1 insertion(+), 4 deletions(-) diff --git a/pinot-core/src/main/java/org/apache/pinot/core/plan/GlobalPlanImplV0.java b/pinot-core/src/main/java/org/apache/pinot/core/plan/GlobalPlanImplV0.java index 073b1cae68b..94cccc45ae9 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/plan/GlobalPlanImplV0.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/plan/GlobalPlanImplV0.java @@ -19,7 +19,6 @@ package org.apache.pinot.core.plan; import java.util.concurrent.TimeoutException; -import org.apache.pinot.common.datatable.DataTable; import org.apache.pinot.core.operator.InstanceResponseOperator; import org.apache.pinot.core.operator.blocks.InstanceResponseBlock; import org.slf4j.Logger; @@ -48,7 +47,6 @@ public PlanNode getPlanNode() { @Override public InstanceResponseBlock execute() throws TimeoutException { - // TODO(egalpin) add code to add the table name meta data here long startTime = System.currentTimeMillis(); InstanceResponseOperator instanceResponseOperator = _instanceResponsePlanNode.run(); long endTime1 = System.currentTimeMillis(); @@ -59,8 +57,6 @@ public InstanceResponseBlock execute() InstanceResponseBlock instanceResponseBlock = instanceResponseOperator.nextBlock(); long endTime2 = System.currentTimeMillis(); LOGGER.debug("InstanceResponseOperator.nextBlock() took: {}ms", endTime2 - endTime1); - instanceResponseBlock.addMetadata(DataTable.MetadataKey.TABLE.getName(), - _instanceResponsePlanNode._queryContext.getTableName()); return instanceResponseBlock; } } diff --git a/pinot-core/src/main/java/org/apache/pinot/core/query/scheduler/QueryScheduler.java b/pinot-core/src/main/java/org/apache/pinot/core/query/scheduler/QueryScheduler.java index 97a250ea110..607028b7fea 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/query/scheduler/QueryScheduler.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/query/scheduler/QueryScheduler.java @@ -155,6 +155,7 @@ protected byte[] processQueryAndSerialize(ServerQueryRequest queryRequest, Execu Map responseMetadata = instanceResponse.getResponseMetadata(); responseMetadata.put(MetadataKey.REQUEST_ID.getName(), Long.toString(requestId)); responseMetadata.put(MetadataKey.QUERY_HASH.getName(), Integer.toString(queryRequest.getQueryHash())); + responseMetadata.put(MetadataKey.TABLE.getName(), queryRequest.getTableNameWithType()); byte[] responseBytes = serializeResponse(queryRequest, instanceResponse); // Log the statistics From 7cac31013afb70a7cb4c5dba0075b50f45200d56 Mon Sep 17 00:00:00 2001 From: egalpin Date: Wed, 7 Aug 2024 16:17:42 -0700 Subject: [PATCH 12/49] Duplicate addition of table name? --- .../main/java/org/apache/pinot/core/plan/GlobalPlanImplV0.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/pinot-core/src/main/java/org/apache/pinot/core/plan/GlobalPlanImplV0.java b/pinot-core/src/main/java/org/apache/pinot/core/plan/GlobalPlanImplV0.java index 94cccc45ae9..0ef539758d2 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/plan/GlobalPlanImplV0.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/plan/GlobalPlanImplV0.java @@ -19,6 +19,7 @@ package org.apache.pinot.core.plan; import java.util.concurrent.TimeoutException; +import org.apache.pinot.common.datatable.DataTable; import org.apache.pinot.core.operator.InstanceResponseOperator; import org.apache.pinot.core.operator.blocks.InstanceResponseBlock; import org.slf4j.Logger; @@ -57,6 +58,8 @@ public InstanceResponseBlock execute() InstanceResponseBlock instanceResponseBlock = instanceResponseOperator.nextBlock(); long endTime2 = System.currentTimeMillis(); LOGGER.debug("InstanceResponseOperator.nextBlock() took: {}ms", endTime2 - endTime1); + instanceResponseBlock.addMetadata(DataTable.MetadataKey.TABLE.getName(), + _instanceResponsePlanNode._queryContext.getTableName()); return instanceResponseBlock; } } From c41813a2c0b1445d84c14482ba918117edcd2032 Mon Sep 17 00:00:00 2001 From: egalpin Date: Thu, 8 Aug 2024 11:32:28 -0700 Subject: [PATCH 13/49] Ensures async query response map employs ConcurrentHashMap at the nested level --- .../apache/pinot/core/transport/AsyncQueryResponse.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/pinot-core/src/main/java/org/apache/pinot/core/transport/AsyncQueryResponse.java b/pinot-core/src/main/java/org/apache/pinot/core/transport/AsyncQueryResponse.java index 8d94a6aadcc..33e01e1221c 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/transport/AsyncQueryResponse.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/transport/AsyncQueryResponse.java @@ -43,7 +43,7 @@ public class AsyncQueryResponse implements QueryResponse { private final long _requestId; private final AtomicReference _status = new AtomicReference<>(Status.IN_PROGRESS); private final AtomicInteger _numServersResponded = new AtomicInteger(); - private final Map> _responses; + private final ConcurrentHashMap> _responses; private final CountDownLatch _countDownLatch; private final long _maxEndTimeMs; private final long _timeoutMs; @@ -68,7 +68,7 @@ public AsyncQueryResponse(QueryRouter queryRouter, long requestId, // submission stats. _serverRoutingStatsManager.recordStatsForQuerySubmission(requestId, serverRequests.getKey().getInstanceId()); - _responses.computeIfAbsent(serverRequests.getKey(), k -> new HashMap<>()) + _responses.computeIfAbsent(serverRequests.getKey(), k -> new ConcurrentHashMap<>()) // we use query hash so that the same hash ID can be passed back from servers more easily than trying to // instantiate a valid InstanceRequest obj and send its hash .put(request.getQuery().getPinotQuery().hashCode(), new ServerResponse(startTimeMs)); @@ -93,7 +93,7 @@ public int getNumServersResponded() { private Map> getFlatResponses() { Map> flattened = new HashMap<>(); - for (Map.Entry> serverResponses : _responses.entrySet()) { + for (Map.Entry> serverResponses : _responses.entrySet()) { ServerRoutingInstance serverRoutingInstance = serverResponses.getKey(); for (ServerResponse serverResponse : serverResponses.getValue().values()) { @@ -138,7 +138,7 @@ public Map> getFinalResponses() public String getServerStats() { StringBuilder stringBuilder = new StringBuilder( "(Server=SubmitDelayMs,ResponseDelayMs,ResponseSize,DeserializationTimeMs,RequestSentDelayMs)"); - for (Map.Entry> serverResponses : _responses.entrySet()) { + for (Map.Entry> serverResponses : _responses.entrySet()) { for (Map.Entry responsePair : serverResponses.getValue().entrySet()) { ServerRoutingInstance serverRoutingInstance = serverResponses.getKey(); stringBuilder.append(';').append(serverRoutingInstance.getShortName()).append('=') From df17781efabda4be18cd931b65be574f32766f11 Mon Sep 17 00:00:00 2001 From: egalpin Date: Thu, 8 Aug 2024 13:11:31 -0700 Subject: [PATCH 14/49] WIP - fixing tests where tablename with type is now required --- .../executor/ServerQueryExecutorV1Impl.java | 33 ++++++++++++------- .../query/reduce/BrokerReduceServiceTest.java | 5 ++- .../transport/ServerRoutingInstanceTest.java | 2 +- .../apache/pinot/queries/BaseQueriesTest.java | 1 + ...egmentSelectionSingleValueQueriesTest.java | 2 +- 5 files changed, 28 insertions(+), 15 deletions(-) diff --git a/pinot-core/src/main/java/org/apache/pinot/core/query/executor/ServerQueryExecutorV1Impl.java b/pinot-core/src/main/java/org/apache/pinot/core/query/executor/ServerQueryExecutorV1Impl.java index 8c2906db541..e09cb318c56 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/query/executor/ServerQueryExecutorV1Impl.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/query/executor/ServerQueryExecutorV1Impl.java @@ -18,6 +18,7 @@ */ package org.apache.pinot.core.query.executor; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import java.util.ArrayList; import java.util.Collections; @@ -133,20 +134,26 @@ public synchronized void shutDown() { @Override public InstanceResponseBlock execute(ServerQueryRequest queryRequest, ExecutorService executorService, @Nullable ResultsBlockStreamer streamer) { + InstanceResponseBlock responseBlock; + if (!queryRequest.isEnableTrace()) { - return executeInternal(queryRequest, executorService, streamer); - } - try { - 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, streamer); - } finally { - Tracing.getTracer().unregister(); + responseBlock = executeInternal(queryRequest, executorService, streamer); + } else { + try { + 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); + responseBlock = executeInternal(queryRequest, executorService, streamer); + } finally { + Tracing.getTracer().unregister(); + } } + + responseBlock.addMetadata(MetadataKey.TABLE.getName(), queryRequest.getTableNameWithType()); + return responseBlock; } private InstanceResponseBlock executeInternal(ServerQueryRequest queryRequest, ExecutorService executorService, @@ -476,6 +483,7 @@ private static Map> getAllSegmentsUniqueExplainPl return operatorDepthToRowDataMap; } + @VisibleForTesting public static InstanceResponseBlock executeExplainQuery(Plan queryPlan, QueryContext queryContext) { ExplainResultsBlock explainResults = new ExplainResultsBlock(queryContext); InstanceResponseOperator responseOperator = (InstanceResponseOperator) queryPlan.getPlanNode().run(); @@ -518,6 +526,7 @@ public static InstanceResponseBlock executeExplainQuery(Plan queryPlan, QueryCon String.valueOf(numEmptyFilterSegments)); instanceResponse.addMetadata(MetadataKey.EXPLAIN_PLAN_NUM_MATCH_ALL_FILTER_SEGMENTS.getName(), String.valueOf(numMatchAllFilterSegments)); + instanceResponse.addMetadata(MetadataKey.TABLE.getName(), queryContext.getTableName()); return instanceResponse; } finally { responseOperator.releaseAll(); diff --git a/pinot-core/src/test/java/org/apache/pinot/core/query/reduce/BrokerReduceServiceTest.java b/pinot-core/src/test/java/org/apache/pinot/core/query/reduce/BrokerReduceServiceTest.java index 9513d92a273..66fe14ba315 100644 --- a/pinot-core/src/test/java/org/apache/pinot/core/query/reduce/BrokerReduceServiceTest.java +++ b/pinot-core/src/test/java/org/apache/pinot/core/query/reduce/BrokerReduceServiceTest.java @@ -37,6 +37,7 @@ import org.apache.pinot.core.transport.ServerRoutingInstance; import org.apache.pinot.spi.env.PinotConfiguration; import org.apache.pinot.spi.utils.CommonConstants.Broker; +import org.apache.pinot.spi.utils.builder.TableNameBuilder; import org.apache.pinot.sql.parsers.CalciteSqlCompiler; import org.testng.annotations.Test; @@ -51,8 +52,9 @@ public void testReduceTimeout() throws IOException { BrokerReduceService brokerReduceService = new BrokerReduceService(new PinotConfiguration(Map.of(Broker.CONFIG_OF_MAX_REDUCE_THREADS_PER_QUERY, 2))); + String tableName = TableNameBuilder.OFFLINE.tableNameWithType("testTable"); BrokerRequest brokerRequest = - CalciteSqlCompiler.compileToBrokerRequest("SELECT COUNT(*) FROM testTable GROUP BY col1"); + CalciteSqlCompiler.compileToBrokerRequest("SELECT COUNT(*) FROM " + tableName + " GROUP BY col1"); DataSchema dataSchema = new DataSchema(new String[]{"col1", "count(*)"}, new ColumnDataType[]{ColumnDataType.INT, ColumnDataType.LONG}); DataTableBuilder dataTableBuilder = DataTableBuilderFactory.getDataTableBuilder(dataSchema); @@ -64,6 +66,7 @@ public void testReduceTimeout() dataTableBuilder.finishRow(); } DataTable dataTable = dataTableBuilder.build(); + dataTable.getMetadata().put(DataTable.MetadataKey.TABLE.getName(), tableName); Map> dataTableMap = new HashMap<>(); int numInstances = 1000; for (int i = 0; i < numInstances; i++) { diff --git a/pinot-core/src/test/java/org/apache/pinot/core/transport/ServerRoutingInstanceTest.java b/pinot-core/src/test/java/org/apache/pinot/core/transport/ServerRoutingInstanceTest.java index 991cd19181e..84c67bff74f 100644 --- a/pinot-core/src/test/java/org/apache/pinot/core/transport/ServerRoutingInstanceTest.java +++ b/pinot-core/src/test/java/org/apache/pinot/core/transport/ServerRoutingInstanceTest.java @@ -27,6 +27,6 @@ public class ServerRoutingInstanceTest { @Test public void equalsVerifier() { EqualsVerifier.configure().forClass(ServerRoutingInstance.class) - .withOnlyTheseFields("_hostname", "_port", "_tableType").suppress(Warning.NULL_FIELDS).verify(); + .withOnlyTheseFields("_hostname", "_port").suppress(Warning.NULL_FIELDS).verify(); } } diff --git a/pinot-core/src/test/java/org/apache/pinot/queries/BaseQueriesTest.java b/pinot-core/src/test/java/org/apache/pinot/queries/BaseQueriesTest.java index ad9eed8f757..f5b430bb2b7 100644 --- a/pinot-core/src/test/java/org/apache/pinot/queries/BaseQueriesTest.java +++ b/pinot-core/src/test/java/org/apache/pinot/queries/BaseQueriesTest.java @@ -61,6 +61,7 @@ import org.apache.pinot.spi.utils.CommonConstants.Broker; import org.apache.pinot.spi.utils.CommonConstants.Server; import org.apache.pinot.spi.utils.ReadMode; +import org.apache.pinot.spi.utils.builder.TableNameBuilder; import org.apache.pinot.sql.parsers.CalciteSqlCompiler; import org.apache.pinot.sql.parsers.CalciteSqlParser; diff --git a/pinot-core/src/test/java/org/apache/pinot/queries/InnerSegmentSelectionSingleValueQueriesTest.java b/pinot-core/src/test/java/org/apache/pinot/queries/InnerSegmentSelectionSingleValueQueriesTest.java index 92d24fc72c0..2a60dae8b43 100644 --- a/pinot-core/src/test/java/org/apache/pinot/queries/InnerSegmentSelectionSingleValueQueriesTest.java +++ b/pinot-core/src/test/java/org/apache/pinot/queries/InnerSegmentSelectionSingleValueQueriesTest.java @@ -597,7 +597,7 @@ Map computeColumnNameToIndexMap(DataSchema dataSchema) { @Test public void testThreadCpuTime() { - String query = "SELECT * FROM testTable"; + String query = "SELECT * FROM testTable_OFFLINE"; ThreadResourceUsageProvider.setThreadCpuTimeMeasurementEnabled(true); // NOTE: Need to check whether thread CPU time measurement is enabled because some environments might not support From 9088b4672bd50b9f3ea4b475074d20ef3b49eb85 Mon Sep 17 00:00:00 2001 From: egalpin Date: Thu, 8 Aug 2024 15:15:34 -0700 Subject: [PATCH 15/49] Removing unnecessary table name metadata set ops --- .../pinot/core/plan/GlobalPlanImplV0.java | 4 +-- .../executor/ServerQueryExecutorV1Impl.java | 34 +++++++------------ .../reduce/ExecutionStatsAggregator.java | 6 ++-- 3 files changed, 19 insertions(+), 25 deletions(-) diff --git a/pinot-core/src/main/java/org/apache/pinot/core/plan/GlobalPlanImplV0.java b/pinot-core/src/main/java/org/apache/pinot/core/plan/GlobalPlanImplV0.java index 0ef539758d2..e5ced4c4943 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/plan/GlobalPlanImplV0.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/plan/GlobalPlanImplV0.java @@ -58,8 +58,8 @@ public InstanceResponseBlock execute() InstanceResponseBlock instanceResponseBlock = instanceResponseOperator.nextBlock(); long endTime2 = System.currentTimeMillis(); LOGGER.debug("InstanceResponseOperator.nextBlock() took: {}ms", endTime2 - endTime1); - instanceResponseBlock.addMetadata(DataTable.MetadataKey.TABLE.getName(), - _instanceResponsePlanNode._queryContext.getTableName()); +// instanceResponseBlock.addMetadata(DataTable.MetadataKey.TABLE.getName(), +// _instanceResponsePlanNode._queryContext.getTableName()); return instanceResponseBlock; } } diff --git a/pinot-core/src/main/java/org/apache/pinot/core/query/executor/ServerQueryExecutorV1Impl.java b/pinot-core/src/main/java/org/apache/pinot/core/query/executor/ServerQueryExecutorV1Impl.java index e09cb318c56..c1e176f9c99 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/query/executor/ServerQueryExecutorV1Impl.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/query/executor/ServerQueryExecutorV1Impl.java @@ -18,7 +18,6 @@ */ package org.apache.pinot.core.query.executor; -import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import java.util.ArrayList; import java.util.Collections; @@ -134,26 +133,20 @@ public synchronized void shutDown() { @Override public InstanceResponseBlock execute(ServerQueryRequest queryRequest, ExecutorService executorService, @Nullable ResultsBlockStreamer streamer) { - InstanceResponseBlock responseBlock; - if (!queryRequest.isEnableTrace()) { - responseBlock = executeInternal(queryRequest, executorService, streamer); - } else { - try { - 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); - responseBlock = executeInternal(queryRequest, executorService, streamer); - } finally { - Tracing.getTracer().unregister(); - } + return executeInternal(queryRequest, executorService, streamer); + } + try { + 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, streamer); + } finally { + Tracing.getTracer().unregister(); } - - responseBlock.addMetadata(MetadataKey.TABLE.getName(), queryRequest.getTableNameWithType()); - return responseBlock; } private InstanceResponseBlock executeInternal(ServerQueryRequest queryRequest, ExecutorService executorService, @@ -483,7 +476,6 @@ private static Map> getAllSegmentsUniqueExplainPl return operatorDepthToRowDataMap; } - @VisibleForTesting public static InstanceResponseBlock executeExplainQuery(Plan queryPlan, QueryContext queryContext) { ExplainResultsBlock explainResults = new ExplainResultsBlock(queryContext); InstanceResponseOperator responseOperator = (InstanceResponseOperator) queryPlan.getPlanNode().run(); @@ -526,7 +518,7 @@ public static InstanceResponseBlock executeExplainQuery(Plan queryPlan, QueryCon String.valueOf(numEmptyFilterSegments)); instanceResponse.addMetadata(MetadataKey.EXPLAIN_PLAN_NUM_MATCH_ALL_FILTER_SEGMENTS.getName(), String.valueOf(numMatchAllFilterSegments)); - instanceResponse.addMetadata(MetadataKey.TABLE.getName(), queryContext.getTableName()); +// instanceResponse.addMetadata(MetadataKey.TABLE.getName(), queryContext.getTableName()); return instanceResponse; } finally { responseOperator.releaseAll(); diff --git a/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/ExecutionStatsAggregator.java b/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/ExecutionStatsAggregator.java index 4f3d71ca31e..18f7739e407 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/ExecutionStatsAggregator.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/ExecutionStatsAggregator.java @@ -74,8 +74,10 @@ public ExecutionStatsAggregator(boolean enableTrace) { public void aggregate(ServerRoutingInstance serverRoutingInstance, DataTable dataTable) { String instanceName = serverRoutingInstance.getShortName(); Map metadata = dataTable.getMetadata(); - TableType tableType = - TableNameBuilder.getTableTypeFromTableName(metadata.get(DataTable.MetadataKey.TABLE.getName())); + TableType tableType = null; + if (metadata.get(DataTable.MetadataKey.TABLE.getName()) != null) { + tableType = TableNameBuilder.getTableTypeFromTableName(metadata.get(DataTable.MetadataKey.TABLE.getName())); + } // Reduce on trace info. if (_enableTrace && metadata.containsKey(DataTable.MetadataKey.TRACE_INFO.getName())) { From 3124c99ebee23db89583e5eb44000077931f0cae Mon Sep 17 00:00:00 2001 From: egalpin Date: Thu, 8 Aug 2024 15:17:49 -0700 Subject: [PATCH 16/49] Updates more tests to be compatible --- .../org/apache/pinot/queries/BaseQueriesTest.java | 11 ++++++++--- .../InnerSegmentSelectionSingleValueQueriesTest.java | 2 +- .../tests/HybridClusterIntegrationTest.java | 10 ++++------ 3 files changed, 13 insertions(+), 10 deletions(-) diff --git a/pinot-core/src/test/java/org/apache/pinot/queries/BaseQueriesTest.java b/pinot-core/src/test/java/org/apache/pinot/queries/BaseQueriesTest.java index f5b430bb2b7..9ee78c2152c 100644 --- a/pinot-core/src/test/java/org/apache/pinot/queries/BaseQueriesTest.java +++ b/pinot-core/src/test/java/org/apache/pinot/queries/BaseQueriesTest.java @@ -225,11 +225,16 @@ private BrokerResponseNative getBrokerResponse(PinotQuery pinotQuery, PlanMaker Map> dataTableMap = new HashMap<>(); try { // For multi-threaded BrokerReduceService, we cannot reuse the same data-table - byte[] serializedResponse = instanceResponse.toDataTable().toBytes(); ServerRoutingInstance serverRoutingInstance = new ServerRoutingInstance("localhost", 1234); dataTableMap.put(serverRoutingInstance, new ArrayList<>()); - dataTableMap.get(serverRoutingInstance).add(DataTableFactory.getDataTable(serializedResponse)); - dataTableMap.get(serverRoutingInstance).add(DataTableFactory.getDataTable(serializedResponse)); + + DataTable dataTable = instanceResponse.toDataTable(); + dataTable.getMetadata().put(DataTable.MetadataKey.TABLE.getName(), + TableNameBuilder.OFFLINE.tableNameWithType(serverQueryContext.getTableName())); + dataTableMap.get(serverRoutingInstance).add(DataTableFactory.getDataTable(dataTable.toBytes())); + dataTable.getMetadata().put(DataTable.MetadataKey.TABLE.getName(), + TableNameBuilder.REALTIME.tableNameWithType(serverQueryContext.getTableName())); + dataTableMap.get(serverRoutingInstance).add(DataTableFactory.getDataTable(dataTable.toBytes())); } catch (Exception e) { throw new RuntimeException(e); } diff --git a/pinot-core/src/test/java/org/apache/pinot/queries/InnerSegmentSelectionSingleValueQueriesTest.java b/pinot-core/src/test/java/org/apache/pinot/queries/InnerSegmentSelectionSingleValueQueriesTest.java index 2a60dae8b43..92d24fc72c0 100644 --- a/pinot-core/src/test/java/org/apache/pinot/queries/InnerSegmentSelectionSingleValueQueriesTest.java +++ b/pinot-core/src/test/java/org/apache/pinot/queries/InnerSegmentSelectionSingleValueQueriesTest.java @@ -597,7 +597,7 @@ Map computeColumnNameToIndexMap(DataSchema dataSchema) { @Test public void testThreadCpuTime() { - String query = "SELECT * FROM testTable_OFFLINE"; + String query = "SELECT * FROM testTable"; ThreadResourceUsageProvider.setThreadCpuTimeMeasurementEnabled(true); // NOTE: Need to check whether thread CPU time measurement is enabled because some environments might not support diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/HybridClusterIntegrationTest.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/HybridClusterIntegrationTest.java index c7a4afc51ab..fba5749c95c 100644 --- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/HybridClusterIntegrationTest.java +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/HybridClusterIntegrationTest.java @@ -284,9 +284,8 @@ public void testQueryTracing(boolean useMultiStageQueryEngine) Assert.assertEquals(jsonNode.get("resultTable").get("rows").get(0).get(0).asLong(), getCountStarResult()); Assert.assertTrue(jsonNode.get("exceptions").isEmpty()); JsonNode traceInfo = jsonNode.get("traceInfo"); - Assert.assertEquals(traceInfo.size(), 2); - Assert.assertTrue(traceInfo.has("localhost_O")); - Assert.assertTrue(traceInfo.has("localhost_R")); + Assert.assertEquals(traceInfo.size(), 1); + Assert.assertTrue(traceInfo.has("localhost")); } @Test(dataProvider = "useBothQueryEngines") @@ -305,9 +304,8 @@ public void testQueryTracingWithLiteral(boolean useMultiStageQueryEngine) } Assert.assertTrue(jsonNode.get("exceptions").isEmpty()); JsonNode traceInfo = jsonNode.get("traceInfo"); - Assert.assertEquals(traceInfo.size(), 2); - Assert.assertTrue(traceInfo.has("localhost_O")); - Assert.assertTrue(traceInfo.has("localhost_R")); + Assert.assertEquals(traceInfo.size(), 1); + Assert.assertTrue(traceInfo.has("localhost")); } @Test(dataProvider = "useBothQueryEngines") From 31a19c7bd3edbba32ce6f755385fb8380bb21b9a Mon Sep 17 00:00:00 2001 From: egalpin Date: Thu, 8 Aug 2024 15:36:09 -0700 Subject: [PATCH 17/49] Ensures query_hash is set everywhere query_id is --- .../org/apache/pinot/core/query/scheduler/QueryScheduler.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/pinot-core/src/main/java/org/apache/pinot/core/query/scheduler/QueryScheduler.java b/pinot-core/src/main/java/org/apache/pinot/core/query/scheduler/QueryScheduler.java index 607028b7fea..6da46b66511 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/query/scheduler/QueryScheduler.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/query/scheduler/QueryScheduler.java @@ -177,6 +177,7 @@ protected byte[] processQueryAndSerialize(ServerQueryRequest queryRequest, Execu instanceResponse = new InstanceResponseBlock(); instanceResponse.addException(QueryException.getException(QueryException.QUERY_CANCELLATION_ERROR, errMsg)); instanceResponse.addMetadata(MetadataKey.REQUEST_ID.getName(), Long.toString(requestId)); + instanceResponse.addMetadata(MetadataKey.QUERY_HASH.getName(), Integer.toString(queryRequest.getQueryHash())); responseBytes = serializeResponse(queryRequest, instanceResponse); } @@ -229,6 +230,7 @@ private byte[] serializeResponse(ServerQueryRequest queryRequest, InstanceRespon LOGGER.error(errMsg); instanceResponse = new InstanceResponseBlock(new ExceptionResultsBlock(new QueryCancelledException(errMsg, e))); instanceResponse.addMetadata(MetadataKey.REQUEST_ID.getName(), Long.toString(queryRequest.getRequestId())); + instanceResponse.addMetadata(MetadataKey.QUERY_HASH.getName(), Integer.toString(queryRequest.getQueryHash())); return serializeResponse(queryRequest, instanceResponse); } catch (Exception e) { _serverMetrics.addMeteredGlobalValue(ServerMeter.RESPONSE_SERIALIZATION_EXCEPTIONS, 1); @@ -251,6 +253,7 @@ protected ListenableFuture immediateErrorResponse(ServerQueryRequest que ProcessingException error) { InstanceResponseBlock instanceResponse = new InstanceResponseBlock(); instanceResponse.addMetadata(MetadataKey.REQUEST_ID.getName(), Long.toString(queryRequest.getRequestId())); + instanceResponse.addMetadata(MetadataKey.QUERY_HASH.getName(), Integer.toString(queryRequest.getQueryHash())); instanceResponse.addException(error); return Futures.immediateFuture(serializeResponse(queryRequest, instanceResponse)); } From 343104a0e381bcbf07c12ce779101aa7e19f7bd0 Mon Sep 17 00:00:00 2001 From: egalpin Date: Thu, 8 Aug 2024 15:43:42 -0700 Subject: [PATCH 18/49] Linting fixes --- .../java/org/apache/pinot/core/plan/GlobalPlanImplV0.java | 1 - .../org/apache/pinot/core/transport/AsyncQueryResponse.java | 6 ++++-- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/pinot-core/src/main/java/org/apache/pinot/core/plan/GlobalPlanImplV0.java b/pinot-core/src/main/java/org/apache/pinot/core/plan/GlobalPlanImplV0.java index e5ced4c4943..444442bf859 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/plan/GlobalPlanImplV0.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/plan/GlobalPlanImplV0.java @@ -19,7 +19,6 @@ package org.apache.pinot.core.plan; import java.util.concurrent.TimeoutException; -import org.apache.pinot.common.datatable.DataTable; import org.apache.pinot.core.operator.InstanceResponseOperator; import org.apache.pinot.core.operator.blocks.InstanceResponseBlock; import org.slf4j.Logger; diff --git a/pinot-core/src/main/java/org/apache/pinot/core/transport/AsyncQueryResponse.java b/pinot-core/src/main/java/org/apache/pinot/core/transport/AsyncQueryResponse.java index 33e01e1221c..5fc0edbc962 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/transport/AsyncQueryResponse.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/transport/AsyncQueryResponse.java @@ -93,7 +93,8 @@ public int getNumServersResponded() { private Map> getFlatResponses() { Map> flattened = new HashMap<>(); - for (Map.Entry> serverResponses : _responses.entrySet()) { + for (Map.Entry> serverResponses + : _responses.entrySet()) { ServerRoutingInstance serverRoutingInstance = serverResponses.getKey(); for (ServerResponse serverResponse : serverResponses.getValue().values()) { @@ -138,7 +139,8 @@ public Map> getFinalResponses() public String getServerStats() { StringBuilder stringBuilder = new StringBuilder( "(Server=SubmitDelayMs,ResponseDelayMs,ResponseSize,DeserializationTimeMs,RequestSentDelayMs)"); - for (Map.Entry> serverResponses : _responses.entrySet()) { + for (Map.Entry> serverResponses + : _responses.entrySet()) { for (Map.Entry responsePair : serverResponses.getValue().entrySet()) { ServerRoutingInstance serverRoutingInstance = serverResponses.getKey(); stringBuilder.append(';').append(serverRoutingInstance.getShortName()).append('=') From 6248be07b4c48df8c19914c5a3ec7c851a8fc077 Mon Sep 17 00:00:00 2001 From: egalpin Date: Fri, 9 Aug 2024 10:48:39 -0700 Subject: [PATCH 19/49] Fixes bug in broker reduce service, remove empty serverInstance entries --- .../pinot/core/query/reduce/BrokerReduceService.java | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/BrokerReduceService.java b/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/BrokerReduceService.java index ee68f1bdad5..da1d5b00b0f 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/BrokerReduceService.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/BrokerReduceService.java @@ -118,6 +118,13 @@ public BrokerResponseNative reduceOnDataTable(BrokerRequest brokerRequest, Broke } } + // Remove map entries where there are no longer any associated data tables (removed by above) + for (Map.Entry> serverResponses : dataTableMap.entrySet()) { + if (serverResponses.getValue().isEmpty()) { + dataTableMap.remove(serverResponses.getKey()); + } + } + String tableName = serverBrokerRequest.getQuerySource().getTableName(); String rawTableName = TableNameBuilder.extractRawTableName(tableName); From 30933ac6bfece7f64e680b9c199b59bcbc8fa96f Mon Sep 17 00:00:00 2001 From: egalpin Date: Fri, 9 Aug 2024 11:12:13 -0700 Subject: [PATCH 20/49] Updates data table reducers to check size number of data tables as opposed to map entries --- .../query/reduce/AggregationDataTableReducer.java | 8 ++++---- .../core/query/reduce/BrokerReduceService.java | 13 ++++++++----- .../query/reduce/DistinctDataTableReducer.java | 14 ++++++++------ .../query/reduce/ExplainPlanDataTableReducer.java | 15 ++++++++------- .../query/reduce/GroupByDataTableReducer.java | 4 ++-- .../query/reduce/SelectionDataTableReducer.java | 8 +++++--- 6 files changed, 35 insertions(+), 27 deletions(-) diff --git a/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/AggregationDataTableReducer.java b/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/AggregationDataTableReducer.java index 7e5dc22c977..b59422d6e16 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/AggregationDataTableReducer.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/AggregationDataTableReducer.java @@ -61,16 +61,16 @@ public void reduceAndSetResults(String tableName, DataSchema dataSchema, DataTableReducerContext reducerContext, BrokerMetrics brokerMetrics) { dataSchema = ReducerDataSchemaUtils.canonicalizeDataSchemaForAggregation(_queryContext, dataSchema); - if (dataTableMap.isEmpty()) { + // flatten the data tables from all servers responding to the query into a single collection of DataTables + Collection dataTables = getFlatDataTables(dataTableMap); + + if (dataTables.isEmpty()) { DataSchema resultTableSchema = new PostAggregationHandler(_queryContext, getPrePostAggregationDataSchema(dataSchema)).getResultDataSchema(); brokerResponseNative.setResultTable(new ResultTable(resultTableSchema, Collections.emptyList())); return; } - // flatten the data tables from all servers responding to the query into a single collection of DataTables - Collection dataTables = getFlatDataTables(dataTableMap); - if (_queryContext.isServerReturnFinalResult()) { if (dataTables.size() == 1) { processSingleFinalResult(dataSchema, dataTables.iterator().next(), brokerResponseNative); diff --git a/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/BrokerReduceService.java b/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/BrokerReduceService.java index da1d5b00b0f..528448e392e 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/BrokerReduceService.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/BrokerReduceService.java @@ -116,15 +116,18 @@ public BrokerResponseNative reduceOnDataTable(BrokerRequest brokerRequest, Broke } } } - } - - // Remove map entries where there are no longer any associated data tables (removed by above) - for (Map.Entry> serverResponses : dataTableMap.entrySet()) { - if (serverResponses.getValue().isEmpty()) { + // Remove map entries where there are no longer any associated data tables (removed by above) + if (dataTableMap.get(serverResponses.getKey()).isEmpty()) { dataTableMap.remove(serverResponses.getKey()); } } +// for (Map.Entry> serverResponses : dataTableMap.entrySet()) { +// if (serverResponses.getValue().isEmpty()) { +// dataTableMap.remove(serverResponses.getKey()); +// } +// } + String tableName = serverBrokerRequest.getQuerySource().getTableName(); String rawTableName = TableNameBuilder.extractRawTableName(tableName); diff --git a/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/DistinctDataTableReducer.java b/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/DistinctDataTableReducer.java index 59b8a68b6e7..8fd9e9182d9 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/DistinctDataTableReducer.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/DistinctDataTableReducer.java @@ -56,18 +56,20 @@ public void reduceAndSetResults(String tableName, DataSchema dataSchema, DistinctTable distinctTable = new DistinctTable(dataSchema, _queryContext.getOrderByExpressions(), _queryContext.getLimit(), _queryContext.isNullHandlingEnabled()); + + Collection dataTables = getFlatDataTables(dataTableMap); if (distinctTable.hasOrderBy()) { - addToOrderByDistinctTable(dataSchema, dataTableMap, distinctTable); + addToOrderByDistinctTable(dataSchema, dataTables, distinctTable); } else { - addToNonOrderByDistinctTable(dataSchema, dataTableMap, distinctTable); + addToNonOrderByDistinctTable(dataSchema, dataTables, distinctTable); } brokerResponseNative.setResultTable(reduceToResultTable(distinctTable)); } private void addToOrderByDistinctTable(DataSchema dataSchema, - Map> dataTableMap, DistinctTable distinctTable) { + Collection dataTables, DistinctTable distinctTable) { - for (DataTable dataTable : getFlatDataTables(dataTableMap)) { + for (DataTable dataTable : dataTables) { Tracing.ThreadAccountantOps.sampleAndCheckInterruption(); int numColumns = dataSchema.size(); int numRows = dataTable.getNumberOfRows(); @@ -89,8 +91,8 @@ private void addToOrderByDistinctTable(DataSchema dataSchema, } private void addToNonOrderByDistinctTable(DataSchema dataSchema, - Map> dataTableMap, DistinctTable distinctTable) { - for (DataTable dataTable : getFlatDataTables(dataTableMap)) { + Collection dataTables, DistinctTable distinctTable) { + for (DataTable dataTable : dataTables) { Tracing.ThreadAccountantOps.sampleAndCheckInterruption(); int numColumns = dataSchema.size(); int numRows = dataTable.getNumberOfRows(); diff --git a/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/ExplainPlanDataTableReducer.java b/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/ExplainPlanDataTableReducer.java index d41615d2e64..8293f17b237 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/ExplainPlanDataTableReducer.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/ExplainPlanDataTableReducer.java @@ -63,9 +63,10 @@ public void reduceAndSetResults(String tableName, DataSchema dataSchema, // Top node should be a BROKER_REDUCE node. addBrokerReduceOperation(reducedRows); + Collection dataTables = getFlatDataTables(dataTableMap); // Construct the combine node - Object[] combinedRow = extractCombineNode(dataTableMap); + Object[] combinedRow = extractCombineNode(dataTables); if (combinedRow != null) { reducedRows.add(combinedRow); } @@ -75,7 +76,7 @@ public void reduceAndSetResults(String tableName, DataSchema dataSchema, boolean explainPlanVerbose = queryOptions != null && QueryOptionsUtils.isExplainPlanVerbose(queryOptions); // Add the rest of the rows for each unique Explain plan received from the servers - List explainPlanRowsList = extractUniqueExplainPlansAcrossServers(dataTableMap, combinedRow); + List explainPlanRowsList = extractUniqueExplainPlansAcrossServers(dataTables, combinedRow); if (!explainPlanVerbose && (explainPlanRowsList.size() > 1)) { // Pick the most appropriate plan if verbose option is disabled explainPlanRowsList = chooseBestExplainPlanToUse(explainPlanRowsList); @@ -104,13 +105,13 @@ public void reduceAndSetResults(String tableName, DataSchema dataSchema, * Extract the combine node to use as the global combine step if present. If no combine node is found, return null. * A combine node may not be found if all segments were pruned across all servers. */ - private Object[] extractCombineNode(Map> dataTableMap) { - if (dataTableMap.isEmpty()) { + private Object[] extractCombineNode(Collection dataTables) { + if (dataTables.isEmpty()) { return null; } Object[] combineRow = null; - for (DataTable dataTable : getFlatDataTables(dataTableMap)) { + for (DataTable dataTable : dataTables) { int numRows = dataTable.getNumberOfRows(); if (numRows > 0) { // First row should be the combine row data, unless all segments were pruned from the Server side @@ -129,11 +130,11 @@ private Object[] extractCombineNode(Map extractUniqueExplainPlansAcrossServers( - Map> dataTableMap, Object[] combinedRow) { + Collection dataTables, Object[] combinedRow) { List explainPlanRowsList = new ArrayList<>(); HashSet explainPlanHashCodeSet = new HashSet<>(); - for (DataTable dataTable : getFlatDataTables(dataTableMap)) { + for (DataTable dataTable : dataTables) { int numRows = dataTable.getNumberOfRows(); ExplainPlanRows explainPlanRows = null; diff --git a/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/GroupByDataTableReducer.java b/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/GroupByDataTableReducer.java index 669f32cf264..1d24e2c4d3a 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/GroupByDataTableReducer.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/GroupByDataTableReducer.java @@ -105,7 +105,8 @@ public void reduceAndSetResults(String tableName, DataSchema dataSchema, DataTableReducerContext reducerContext, BrokerMetrics brokerMetrics) { dataSchema = ReducerDataSchemaUtils.canonicalizeDataSchemaForGroupBy(_queryContext, dataSchema); - if (dataTableMap.isEmpty()) { + Collection dataTables = getFlatDataTables(dataTableMap); + if (dataTables.isEmpty()) { PostAggregationHandler postAggregationHandler = new PostAggregationHandler(_queryContext, getPrePostAggregationDataSchema(dataSchema)); DataSchema resultDataSchema = postAggregationHandler.getResultDataSchema(); @@ -114,7 +115,6 @@ public void reduceAndSetResults(String tableName, DataSchema dataSchema, return; } - Collection dataTables = getFlatDataTables(dataTableMap); // NOTE: Use regular reduce when group keys are not partitioned even if there are only one data table because the // records are not sorted yet. if (_queryContext.isServerReturnFinalResult() && dataTables.size() == 1) { diff --git a/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/SelectionDataTableReducer.java b/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/SelectionDataTableReducer.java index 825c90a1aec..af57936f894 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/SelectionDataTableReducer.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/SelectionDataTableReducer.java @@ -54,13 +54,15 @@ public void reduceAndSetResults(String tableName, DataSchema dataSchema, Pair pair = SelectionOperatorUtils.getResultTableDataSchemaAndColumnIndices(_queryContext, dataSchema); int limit = _queryContext.getLimit(); - if (dataTableMap.isEmpty() || limit == 0) { + Collection dataTables = getFlatDataTables(dataTableMap); + + if (dataTables.isEmpty() || limit == 0) { brokerResponseNative.setResultTable(new ResultTable(pair.getLeft(), Collections.emptyList())); return; } if (_queryContext.getOrderByExpressions() == null) { // Selection only - List reducedRows = SelectionOperatorUtils.reduceWithoutOrdering(getFlatDataTables(dataTableMap), limit, + List reducedRows = SelectionOperatorUtils.reduceWithoutOrdering(dataTables, limit, _queryContext.isNullHandlingEnabled()); brokerResponseNative.setResultTable( SelectionOperatorUtils.renderResultTableWithoutOrdering(reducedRows, pair.getLeft(), pair.getRight())); @@ -68,7 +70,7 @@ public void reduceAndSetResults(String tableName, DataSchema dataSchema, // Selection order-by SelectionOperatorService selectionService = new SelectionOperatorService(_queryContext, pair.getLeft(), pair.getRight()); - selectionService.reduceWithOrdering(getFlatDataTables(dataTableMap)); + selectionService.reduceWithOrdering(dataTables); brokerResponseNative.setResultTable(selectionService.renderResultTableWithOrdering()); } } From ff55b3057c4a0faf66f0f2eedf66d666ca7e2a8d Mon Sep 17 00:00:00 2001 From: egalpin Date: Fri, 9 Aug 2024 12:59:09 -0700 Subject: [PATCH 21/49] Ensures no concurrent modification of nested map --- .../core/query/reduce/BrokerReduceService.java | 17 ++++++----------- 1 file changed, 6 insertions(+), 11 deletions(-) diff --git a/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/BrokerReduceService.java b/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/BrokerReduceService.java index 528448e392e..d0f195142b8 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/BrokerReduceService.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/BrokerReduceService.java @@ -80,11 +80,12 @@ public BrokerResponseNative reduceOnDataTable(BrokerRequest brokerRequest, Broke List serversWithConflictingDataSchema = new ArrayList<>(); // Process server response metadata. - for (Map.Entry> serverResponses : dataTableMap.entrySet()) { - Iterator tableIter = serverResponses.getValue().iterator(); + Iterator serverIter = dataTableMap.keySet().iterator(); + while (serverIter.hasNext()) { + ServerRoutingInstance serverRoutingInstance = serverIter.next(); + Iterator tableIter = dataTableMap.get(serverRoutingInstance).iterator(); while (tableIter.hasNext()) { DataTable dataTable = tableIter.next(); - ServerRoutingInstance serverRoutingInstance = serverResponses.getKey(); // aggregate metrics aggregator.aggregate(serverRoutingInstance, dataTable); @@ -117,17 +118,11 @@ public BrokerResponseNative reduceOnDataTable(BrokerRequest brokerRequest, Broke } } // Remove map entries where there are no longer any associated data tables (removed by above) - if (dataTableMap.get(serverResponses.getKey()).isEmpty()) { - dataTableMap.remove(serverResponses.getKey()); + if (dataTableMap.get(serverRoutingInstance).isEmpty()) { + serverIter.remove(); } } -// for (Map.Entry> serverResponses : dataTableMap.entrySet()) { -// if (serverResponses.getValue().isEmpty()) { -// dataTableMap.remove(serverResponses.getKey()); -// } -// } - String tableName = serverBrokerRequest.getQuerySource().getTableName(); String rawTableName = TableNameBuilder.extractRawTableName(tableName); From 8c8c0061524169d8eece676b26660eca8033272b Mon Sep 17 00:00:00 2001 From: egalpin Date: Fri, 9 Aug 2024 16:46:26 -0700 Subject: [PATCH 22/49] Adds comment RE deployment --- .../java/org/apache/pinot/core/transport/QueryRouter.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/pinot-core/src/main/java/org/apache/pinot/core/transport/QueryRouter.java b/pinot-core/src/main/java/org/apache/pinot/core/transport/QueryRouter.java index 1d221dc7323..0102a66f2b5 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/transport/QueryRouter.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/transport/QueryRouter.java @@ -182,8 +182,11 @@ public void shutDown() { void receiveDataTable(ServerRoutingInstance serverRoutingInstance, DataTable dataTable, int responseSize, int deserializationTimeMs) { - // TODO(egalpin): read queryHash from dataTable metadata long requestId = Long.parseLong(dataTable.getMetadata().get(MetadataKey.REQUEST_ID.getName())); + + // TODO(egalpin): How can we handle rolling out brokers expecting query_hash to be present, but while old server + // versions are still running and not yet setting the query hash from their side? If possible, deploying servers + // first would work. int queryHash = Integer.parseInt(dataTable.getMetadata().get(MetadataKey.QUERY_HASH.getName())); AsyncQueryResponse asyncQueryResponse = _asyncQueryResponseMap.get(requestId); From 7eac151c012e32d385c2114caba0505489936013 Mon Sep 17 00:00:00 2001 From: egalpin Date: Mon, 12 Aug 2024 13:05:20 -0700 Subject: [PATCH 23/49] Uses count of issued queries rather than distinct server count for setting QueryOptionKey.SERVER_RETURN_FINAL_RESULT --- .../BaseSingleStageBrokerRequestHandler.java | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BaseSingleStageBrokerRequestHandler.java b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BaseSingleStageBrokerRequestHandler.java index 973d4b2567c..3692672aeb6 100644 --- a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BaseSingleStageBrokerRequestHandler.java +++ b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BaseSingleStageBrokerRequestHandler.java @@ -719,13 +719,13 @@ protected BrokerResponse handleRequest(long requestId, String query, @Nullable S // Set the maximum serialized response size per server, and ask server to directly return final response when only // one server is queried - int numServers = queryRoutingTable.size(); + int numQueriesIssued = queryRoutingTable.values().stream().mapToInt(List::size).sum(); if (offlineBrokerRequest != null) { Map queryOptions = offlineBrokerRequest.getPinotQuery().getQueryOptions(); - setMaxServerResponseSizeBytes(numServers, queryOptions, offlineTableConfig); + setMaxServerResponseSizeBytes(numQueriesIssued, queryOptions, offlineTableConfig); // Set the query option to directly return final result for single server query unless it is explicitly disabled - if (numServers == 1) { + if (numQueriesIssued == 1) { // Set the same flag in the original server request to be used in the reduce phase for hybrid table if (queryOptions.putIfAbsent(QueryOptionKey.SERVER_RETURN_FINAL_RESULT, "true") == null && offlineBrokerRequest != serverBrokerRequest) { @@ -736,9 +736,9 @@ protected BrokerResponse handleRequest(long requestId, String query, @Nullable S } if (realtimeBrokerRequest != null) { Map queryOptions = realtimeBrokerRequest.getPinotQuery().getQueryOptions(); - setMaxServerResponseSizeBytes(numServers, queryOptions, realtimeTableConfig); + setMaxServerResponseSizeBytes(numQueriesIssued, queryOptions, realtimeTableConfig); // Set the query option to directly return final result for single server query unless it is explicitly disabled - if (numServers == 1) { + if (numQueriesIssued == 1) { // Set the same flag in the original server request to be used in the reduce phase for hybrid table if (queryOptions.putIfAbsent(QueryOptionKey.SERVER_RETURN_FINAL_RESULT, "true") == null && realtimeBrokerRequest != serverBrokerRequest) { @@ -1813,7 +1813,7 @@ private long setQueryTimeout(String tableNameWithType, Map query * 5. BrokerConfig -> maxServerResponseSizeBytes * 6. BrokerConfig -> maxServerResponseSizeBytes */ - private void setMaxServerResponseSizeBytes(int numServers, Map queryOptions, + private void setMaxServerResponseSizeBytes(int numQueriesIssued, Map queryOptions, @Nullable TableConfig tableConfig) { // QueryOption if (QueryOptionsUtils.getMaxServerResponseSizeBytes(queryOptions) != null) { @@ -1822,7 +1822,7 @@ private void setMaxServerResponseSizeBytes(int numServers, Map q Long maxQueryResponseSizeQueryOption = QueryOptionsUtils.getMaxQueryResponseSizeBytes(queryOptions); if (maxQueryResponseSizeQueryOption != null) { queryOptions.put(QueryOptionKey.MAX_SERVER_RESPONSE_SIZE_BYTES, - Long.toString(maxQueryResponseSizeQueryOption / numServers)); + Long.toString(maxQueryResponseSizeQueryOption / numQueriesIssued)); return; } @@ -1836,7 +1836,7 @@ private void setMaxServerResponseSizeBytes(int numServers, Map q } if (queryConfig.getMaxQueryResponseSizeBytes() != null) { queryOptions.put(QueryOptionKey.MAX_SERVER_RESPONSE_SIZE_BYTES, - Long.toString(queryConfig.getMaxQueryResponseSizeBytes() / numServers)); + Long.toString(queryConfig.getMaxQueryResponseSizeBytes() / numQueriesIssued)); return; } } @@ -1852,7 +1852,7 @@ private void setMaxServerResponseSizeBytes(int numServers, Map q String maxQueryResponseSizeBrokerConfig = _config.getProperty(Broker.CONFIG_OF_MAX_QUERY_RESPONSE_SIZE_BYTES); if (maxQueryResponseSizeBrokerConfig != null) { queryOptions.put(QueryOptionKey.MAX_SERVER_RESPONSE_SIZE_BYTES, - Long.toString(DataSizeUtils.toBytes(maxQueryResponseSizeBrokerConfig) / numServers)); + Long.toString(DataSizeUtils.toBytes(maxQueryResponseSizeBrokerConfig) / numQueriesIssued)); } } From c3d9bf828c35baf36cfd91ef6d9b8f5f4563febd Mon Sep 17 00:00:00 2001 From: egalpin Date: Tue, 13 Aug 2024 16:21:14 -0700 Subject: [PATCH 24/49] Ensures table name is always added to DataTable metadata --- .../org/apache/pinot/core/query/scheduler/QueryScheduler.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/pinot-core/src/main/java/org/apache/pinot/core/query/scheduler/QueryScheduler.java b/pinot-core/src/main/java/org/apache/pinot/core/query/scheduler/QueryScheduler.java index 6da46b66511..c79bc3eab0f 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/query/scheduler/QueryScheduler.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/query/scheduler/QueryScheduler.java @@ -178,6 +178,7 @@ protected byte[] processQueryAndSerialize(ServerQueryRequest queryRequest, Execu instanceResponse.addException(QueryException.getException(QueryException.QUERY_CANCELLATION_ERROR, errMsg)); instanceResponse.addMetadata(MetadataKey.REQUEST_ID.getName(), Long.toString(requestId)); instanceResponse.addMetadata(MetadataKey.QUERY_HASH.getName(), Integer.toString(queryRequest.getQueryHash())); + instanceResponse.addMetadata(MetadataKey.TABLE.getName(), queryRequest.getTableNameWithType()); responseBytes = serializeResponse(queryRequest, instanceResponse); } @@ -231,6 +232,7 @@ private byte[] serializeResponse(ServerQueryRequest queryRequest, InstanceRespon instanceResponse = new InstanceResponseBlock(new ExceptionResultsBlock(new QueryCancelledException(errMsg, e))); instanceResponse.addMetadata(MetadataKey.REQUEST_ID.getName(), Long.toString(queryRequest.getRequestId())); instanceResponse.addMetadata(MetadataKey.QUERY_HASH.getName(), Integer.toString(queryRequest.getQueryHash())); + instanceResponse.addMetadata(MetadataKey.TABLE.getName(), queryRequest.getTableNameWithType()); return serializeResponse(queryRequest, instanceResponse); } catch (Exception e) { _serverMetrics.addMeteredGlobalValue(ServerMeter.RESPONSE_SERIALIZATION_EXCEPTIONS, 1); @@ -254,6 +256,7 @@ protected ListenableFuture immediateErrorResponse(ServerQueryRequest que InstanceResponseBlock instanceResponse = new InstanceResponseBlock(); instanceResponse.addMetadata(MetadataKey.REQUEST_ID.getName(), Long.toString(queryRequest.getRequestId())); instanceResponse.addMetadata(MetadataKey.QUERY_HASH.getName(), Integer.toString(queryRequest.getQueryHash())); + instanceResponse.addMetadata(MetadataKey.TABLE.getName(), queryRequest.getTableNameWithType()); instanceResponse.addException(error); return Futures.immediateFuture(serializeResponse(queryRequest, instanceResponse)); } From 75db44ff3924badd04ed2cb4676060297f5c3f66 Mon Sep 17 00:00:00 2001 From: egalpin Date: Tue, 13 Aug 2024 16:32:25 -0700 Subject: [PATCH 25/49] Fixes use of wrong hashcode --- .../org/apache/pinot/core/transport/AsyncQueryResponse.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pinot-core/src/main/java/org/apache/pinot/core/transport/AsyncQueryResponse.java b/pinot-core/src/main/java/org/apache/pinot/core/transport/AsyncQueryResponse.java index 5fc0edbc962..657f1e69c89 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/transport/AsyncQueryResponse.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/transport/AsyncQueryResponse.java @@ -185,7 +185,7 @@ void markRequestSubmitted(ServerRoutingInstance serverRoutingInstance, InstanceR void markRequestSent(ServerRoutingInstance serverRoutingInstance, InstanceRequest instanceRequest, int requestSentLatencyMs) { - _responses.get(serverRoutingInstance).get(instanceRequest.getQuery().hashCode()) + _responses.get(serverRoutingInstance).get(instanceRequest.getQuery().getPinotQuery().hashCode()) .markRequestSent(requestSentLatencyMs); } From 2ffd65d8f869f5f707af708f1680cbb45fb3a2ea Mon Sep 17 00:00:00 2001 From: egalpin Date: Tue, 13 Aug 2024 16:33:06 -0700 Subject: [PATCH 26/49] Removes codeblock handled elsewhere --- .../BaseSingleStageBrokerRequestHandler.java | 9 +-------- 1 file changed, 1 insertion(+), 8 deletions(-) diff --git a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BaseSingleStageBrokerRequestHandler.java b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BaseSingleStageBrokerRequestHandler.java index 3692672aeb6..bf8807879ef 100644 --- a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BaseSingleStageBrokerRequestHandler.java +++ b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BaseSingleStageBrokerRequestHandler.java @@ -756,14 +756,7 @@ protected BrokerResponse handleRequest(long requestId, String query, @Nullable S // - Compile time function invocation // - Literal only queries // - Any rewrites -// if (pinotQuery.isExplain()) { - // Update routing tables to only send request to offline servers for OFFLINE and HYBRID tables. -// if (offlineRoutingTable != null) { -// // For OFFLINE and HYBRID tables, don't send EXPLAIN query to realtime servers. -// realtimeBrokerRequest = null; -// realtimeRoutingTable = null; -// } -// } + BrokerResponseNative brokerResponse; if (_queriesById != null) { // Start to track the running query for cancellation just before sending it out to servers to avoid any From efc72fbf98a6cbfdd03383545624660cd1d76071 Mon Sep 17 00:00:00 2001 From: egalpin Date: Tue, 3 Sep 2024 16:46:28 -0700 Subject: [PATCH 27/49] WIP - use TableName instead of query hash --- .../query/request/ServerQueryRequest.java | 13 +++---- .../core/query/scheduler/QueryScheduler.java | 4 -- .../core/transport/AsyncQueryResponse.java | 39 +++++++++++-------- .../pinot/core/transport/QueryResponse.java | 15 ++++++- .../pinot/core/transport/QueryRouter.java | 7 +--- .../pinot/core/transport/ServerChannels.java | 3 +- .../core/transport/QueryRoutingTest.java | 2 +- 7 files changed, 46 insertions(+), 37 deletions(-) diff --git a/pinot-core/src/main/java/org/apache/pinot/core/query/request/ServerQueryRequest.java b/pinot-core/src/main/java/org/apache/pinot/core/query/request/ServerQueryRequest.java index 1610db91376..6d6db90cafd 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/query/request/ServerQueryRequest.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/query/request/ServerQueryRequest.java @@ -44,7 +44,7 @@ */ public class ServerQueryRequest { private final long _requestId; - private final int _queryHash; + private final String _tableName; private final String _brokerId; private final boolean _enableTrace; private final boolean _enableStreaming; @@ -72,9 +72,8 @@ public ServerQueryRequest(InstanceRequest instanceRequest, ServerMetrics serverM _segmentsToQuery = instanceRequest.getSearchSegments(); _optionalSegments = instanceRequest.getOptionalSegments(); _queryContext = getQueryContext(instanceRequest.getQuery().getPinotQuery()); - // TODO(egalpin): instanceRequest.getQuery() or instanceRequest.getQuery().getPinotQuery() ? - // needs to match whats in AsyncQueryResponse - _queryHash = instanceRequest.getQuery().getPinotQuery().hashCode(); + // Method to set table name needs to match whats in AsyncQueryResponse + _tableName = instanceRequest.getQuery().getPinotQuery().getDataSource().getTableName(); _queryId = QueryIdUtils.getQueryId(_brokerId, _requestId, TableNameBuilder.getTableTypeFromTableName(_queryContext.getTableName())); _timerContext = new TimerContext(_queryContext.getTableName(), serverMetrics, queryArrivalTimeMs); @@ -106,7 +105,7 @@ public ServerQueryRequest(Server.ServerRequest serverRequest, ServerMetrics serv throw new UnsupportedOperationException("Unsupported payloadType: " + payloadType); } _queryContext = getQueryContext(brokerRequest.getPinotQuery()); - _queryHash = brokerRequest.getPinotQuery().hashCode(); + _tableName = brokerRequest.getPinotQuery().getDataSource().getTableName(); _queryId = QueryIdUtils.getQueryId(_brokerId, _requestId, TableNameBuilder.getTableTypeFromTableName(_queryContext.getTableName())); _timerContext = new TimerContext(_queryContext.getTableName(), serverMetrics, queryArrivalTimeMs); @@ -120,8 +119,8 @@ public long getRequestId() { return _requestId; } - public int getQueryHash() { - return _queryHash; + public String getTableName() { + return _tableName; } public String getBrokerId() { diff --git a/pinot-core/src/main/java/org/apache/pinot/core/query/scheduler/QueryScheduler.java b/pinot-core/src/main/java/org/apache/pinot/core/query/scheduler/QueryScheduler.java index c79bc3eab0f..ec59b211fce 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/query/scheduler/QueryScheduler.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/query/scheduler/QueryScheduler.java @@ -154,7 +154,6 @@ protected byte[] processQueryAndSerialize(ServerQueryRequest queryRequest, Execu long requestId = queryRequest.getRequestId(); Map responseMetadata = instanceResponse.getResponseMetadata(); responseMetadata.put(MetadataKey.REQUEST_ID.getName(), Long.toString(requestId)); - responseMetadata.put(MetadataKey.QUERY_HASH.getName(), Integer.toString(queryRequest.getQueryHash())); responseMetadata.put(MetadataKey.TABLE.getName(), queryRequest.getTableNameWithType()); byte[] responseBytes = serializeResponse(queryRequest, instanceResponse); @@ -177,7 +176,6 @@ protected byte[] processQueryAndSerialize(ServerQueryRequest queryRequest, Execu instanceResponse = new InstanceResponseBlock(); instanceResponse.addException(QueryException.getException(QueryException.QUERY_CANCELLATION_ERROR, errMsg)); instanceResponse.addMetadata(MetadataKey.REQUEST_ID.getName(), Long.toString(requestId)); - instanceResponse.addMetadata(MetadataKey.QUERY_HASH.getName(), Integer.toString(queryRequest.getQueryHash())); instanceResponse.addMetadata(MetadataKey.TABLE.getName(), queryRequest.getTableNameWithType()); responseBytes = serializeResponse(queryRequest, instanceResponse); } @@ -231,7 +229,6 @@ private byte[] serializeResponse(ServerQueryRequest queryRequest, InstanceRespon LOGGER.error(errMsg); instanceResponse = new InstanceResponseBlock(new ExceptionResultsBlock(new QueryCancelledException(errMsg, e))); instanceResponse.addMetadata(MetadataKey.REQUEST_ID.getName(), Long.toString(queryRequest.getRequestId())); - instanceResponse.addMetadata(MetadataKey.QUERY_HASH.getName(), Integer.toString(queryRequest.getQueryHash())); instanceResponse.addMetadata(MetadataKey.TABLE.getName(), queryRequest.getTableNameWithType()); return serializeResponse(queryRequest, instanceResponse); } catch (Exception e) { @@ -255,7 +252,6 @@ protected ListenableFuture immediateErrorResponse(ServerQueryRequest que ProcessingException error) { InstanceResponseBlock instanceResponse = new InstanceResponseBlock(); instanceResponse.addMetadata(MetadataKey.REQUEST_ID.getName(), Long.toString(queryRequest.getRequestId())); - instanceResponse.addMetadata(MetadataKey.QUERY_HASH.getName(), Integer.toString(queryRequest.getQueryHash())); instanceResponse.addMetadata(MetadataKey.TABLE.getName(), queryRequest.getTableNameWithType()); instanceResponse.addException(error); return Futures.immediateFuture(serializeResponse(queryRequest, instanceResponse)); diff --git a/pinot-core/src/main/java/org/apache/pinot/core/transport/AsyncQueryResponse.java b/pinot-core/src/main/java/org/apache/pinot/core/transport/AsyncQueryResponse.java index 657f1e69c89..6b0b27a0895 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/transport/AsyncQueryResponse.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/transport/AsyncQueryResponse.java @@ -43,7 +43,7 @@ public class AsyncQueryResponse implements QueryResponse { private final long _requestId; private final AtomicReference _status = new AtomicReference<>(Status.IN_PROGRESS); private final AtomicInteger _numServersResponded = new AtomicInteger(); - private final ConcurrentHashMap> _responses; + private final ConcurrentHashMap> _responses; private final CountDownLatch _countDownLatch; private final long _maxEndTimeMs; private final long _timeoutMs; @@ -53,7 +53,6 @@ public class AsyncQueryResponse implements QueryResponse { private volatile Exception _exception; public AsyncQueryResponse(QueryRouter queryRouter, long requestId, -// Set serverQueryRoutingContexts, Map> requestMap, long startTimeMs, long timeoutMs, ServerRoutingStatsManager serverRoutingStatsManager) { _queryRouter = queryRouter; @@ -69,9 +68,7 @@ public AsyncQueryResponse(QueryRouter queryRouter, long requestId, _serverRoutingStatsManager.recordStatsForQuerySubmission(requestId, serverRequests.getKey().getInstanceId()); _responses.computeIfAbsent(serverRequests.getKey(), k -> new ConcurrentHashMap<>()) - // we use query hash so that the same hash ID can be passed back from servers more easily than trying to - // instantiate a valid InstanceRequest obj and send its hash - .put(request.getQuery().getPinotQuery().hashCode(), new ServerResponse(startTimeMs)); + .put(request.getQuery().getPinotQuery().getDataSource().getTableName(), new ServerResponse(startTimeMs)); numQueriesIssued++; } } @@ -93,8 +90,8 @@ public int getNumServersResponded() { private Map> getFlatResponses() { Map> flattened = new HashMap<>(); - for (Map.Entry> serverResponses - : _responses.entrySet()) { + for (Map.Entry> serverResponses : + _responses.entrySet()) { ServerRoutingInstance serverRoutingInstance = serverResponses.getKey(); for (ServerResponse serverResponse : serverResponses.getValue().values()) { @@ -106,12 +103,22 @@ private Map> getFlatResponses() { } @Override - public Map> getCurrentResponses() { + public Map getCurrentResponses() { return getFlatResponses(); } @Override - public Map> getFinalResponses() + public Map> getCurrentResponsesPerServer() { + return getFlatResponses(); + } + + @Override + public Map getFinalResponses() { + + } + + @Override + public Map> getFinalResponsesPerServer() throws InterruptedException { Map> flatResponses = getFlatResponses(); try { @@ -139,9 +146,9 @@ public Map> getFinalResponses() public String getServerStats() { StringBuilder stringBuilder = new StringBuilder( "(Server=SubmitDelayMs,ResponseDelayMs,ResponseSize,DeserializationTimeMs,RequestSentDelayMs)"); - for (Map.Entry> serverResponses - : _responses.entrySet()) { - for (Map.Entry responsePair : serverResponses.getValue().entrySet()) { + for (Map.Entry> serverResponses : + _responses.entrySet()) { + for (Map.Entry responsePair : serverResponses.getValue().entrySet()) { ServerRoutingInstance serverRoutingInstance = serverResponses.getKey(); stringBuilder.append(';').append(serverRoutingInstance.getShortName()).append('=') .append(responsePair.getValue().toString()); @@ -189,9 +196,9 @@ void markRequestSent(ServerRoutingInstance serverRoutingInstance, InstanceReques .markRequestSent(requestSentLatencyMs); } - void receiveDataTable(ServerRoutingInstance serverRoutingInstance, int queryHash, DataTable dataTable, + void receiveDataTable(ServerRoutingInstance serverRoutingInstance, String tableName, DataTable dataTable, int responseSize, int deserializationTimeMs) { - ServerResponse response = _responses.get(serverRoutingInstance).get(queryHash); + ServerResponse response = _responses.get(serverRoutingInstance).get(tableName); response.receiveDataTable(dataTable, responseSize, deserializationTimeMs); _countDownLatch.countDown(); _numServersResponded.getAndIncrement(); @@ -212,9 +219,9 @@ void markQueryFailed(ServerRoutingInstance serverRoutingInstance, Exception exce * server hasn't responded yet. */ void markServerDown(ServerRoutingInstance serverRoutingInstance, Exception exception) { - // TODO(egalpin): how to make servers down under the assumption that multiple queries + // TODO(egalpin): how to mark servers down under the assumption that multiple queries // to the same server are valid? - Map serverResponses = _responses.get(serverRoutingInstance); + Map serverResponses = _responses.get(serverRoutingInstance); for (ServerResponse serverResponse : serverResponses.values()) { if (serverResponse != null && serverResponse.getDataTable() == null) { markQueryFailed(serverRoutingInstance, exception); diff --git a/pinot-core/src/main/java/org/apache/pinot/core/transport/QueryResponse.java b/pinot-core/src/main/java/org/apache/pinot/core/transport/QueryResponse.java index caa91a3ff75..72260ade157 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/transport/QueryResponse.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/transport/QueryResponse.java @@ -49,13 +49,24 @@ enum Status { /** * Returns the current server responses without blocking. + * + * @deprecated use {@link #getCurrentResponsesPerServer()} instead. */ - Map> getCurrentResponses(); + @Deprecated(forRemoval = true) + Map getCurrentResponses(); + + Map> getCurrentResponsesPerServer(); /** * Waits until the query is done (COMPLETED, FAILED or TIMED_OUT) and returns the final server responses. + * + * @deprecated use {@link #getFinalResponsesPerServer()} instead. */ - Map> getFinalResponses() + @Deprecated(forRemoval = true) + Map getFinalResponses() + throws InterruptedException; + + Map> getFinalResponsesPerServer() throws InterruptedException; /** diff --git a/pinot-core/src/main/java/org/apache/pinot/core/transport/QueryRouter.java b/pinot-core/src/main/java/org/apache/pinot/core/transport/QueryRouter.java index 0102a66f2b5..6f135b4e827 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/transport/QueryRouter.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/transport/QueryRouter.java @@ -111,9 +111,6 @@ public AsyncQueryResponse submitQuery(long requestId, String rawTableName, skipUnavailableServers = true; } - // TODO(egalpin): Revamp serverRoutingInstance hashCode to include broker request some how? - // TODO(egalpin): This is wrong: there can be many instance requests to the same server i.e. - // serverRoutingInstance requestMap.computeIfAbsent(serverRoutingInstance, k -> new ArrayList<>()).add(instanceRequest); } } @@ -187,12 +184,12 @@ void receiveDataTable(ServerRoutingInstance serverRoutingInstance, DataTable dat // TODO(egalpin): How can we handle rolling out brokers expecting query_hash to be present, but while old server // versions are still running and not yet setting the query hash from their side? If possible, deploying servers // first would work. - int queryHash = Integer.parseInt(dataTable.getMetadata().get(MetadataKey.QUERY_HASH.getName())); + String tableName = dataTable.getMetadata().get(MetadataKey.TABLE.getName()); AsyncQueryResponse asyncQueryResponse = _asyncQueryResponseMap.get(requestId); // Query future might be null if the query is already done (maybe due to failure) if (asyncQueryResponse != null) { - asyncQueryResponse.receiveDataTable(serverRoutingInstance, queryHash, dataTable, responseSize, + asyncQueryResponse.receiveDataTable(serverRoutingInstance, tableName, dataTable, responseSize, deserializationTimeMs); } } diff --git a/pinot-core/src/main/java/org/apache/pinot/core/transport/ServerChannels.java b/pinot-core/src/main/java/org/apache/pinot/core/transport/ServerChannels.java index 03f19b2f2c9..47091d408fe 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/transport/ServerChannels.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/transport/ServerChannels.java @@ -123,9 +123,8 @@ public void sendRequest(String rawTableName, AsyncQueryResponse asyncQueryRespon ServerRoutingInstance serverRoutingInstance, InstanceRequest instanceRequest, long timeoutMs) throws Exception { byte[] requestBytes = _threadLocalTSerializer.get().serialize(instanceRequest); + // TODO(egalpin): create backward compat hash key? _serverToChannelMap.computeIfAbsent(serverRoutingInstance, ServerChannel::new) - // TODO(egalpin): what if serverRoutingInstance was an instance of ServerQueryRoutingContext instead? Use - // ServerRoutingInstance#fromServerQueryContext for the above map key. .sendRequest(rawTableName, asyncQueryResponse, serverRoutingInstance, instanceRequest, requestBytes, timeoutMs); } diff --git a/pinot-core/src/test/java/org/apache/pinot/core/transport/QueryRoutingTest.java b/pinot-core/src/test/java/org/apache/pinot/core/transport/QueryRoutingTest.java index e1490bb97a1..32bde76bfcc 100644 --- a/pinot-core/src/test/java/org/apache/pinot/core/transport/QueryRoutingTest.java +++ b/pinot-core/src/test/java/org/apache/pinot/core/transport/QueryRoutingTest.java @@ -136,7 +136,7 @@ private QueryScheduler mockQueryScheduler(int responseDelayMs, DataTable offline QueryScheduler queryScheduler = mock(QueryScheduler.class); when(queryScheduler.submit(any())).thenAnswer(invocation -> { Thread.sleep(responseDelayMs); - String queryHash = String.valueOf(((ServerQueryRequest) invocation.getArguments()[0]).getQueryHash()); + String queryHash = String.valueOf(((ServerQueryRequest) invocation.getArguments()[0]).getTableName()); if (queryHash.equals(realtimeDataTable.getMetadata().get(MetadataKey.QUERY_HASH.getName()))) { return Futures.immediateFuture(realtimeDataTable.toBytes()); } else if (queryHash.equals(offlineDataTable.getMetadata().get(MetadataKey.QUERY_HASH.getName()))) { From 859311d1ce875283ab74a5ae9e88d4422c18ff75 Mon Sep 17 00:00:00 2001 From: egalpin Date: Fri, 27 Sep 2024 15:06:36 -0700 Subject: [PATCH 28/49] Borrows 1 digit from request-id --- .../BrokerRequestIdGenerator.java | 2 +- .../BrokerRequestIdGeneratorTest.java | 21 +++++++++++++++++++ 2 files changed, 22 insertions(+), 1 deletion(-) create mode 100644 pinot-broker/src/test/java/org/apache/pinot/broker/requesthandler/BrokerRequestIdGeneratorTest.java diff --git a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BrokerRequestIdGenerator.java b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BrokerRequestIdGenerator.java index c97ee44a0af..aee53c180b2 100644 --- a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BrokerRequestIdGenerator.java +++ b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BrokerRequestIdGenerator.java @@ -43,7 +43,7 @@ public BrokerRequestIdGenerator(String brokerId) { } public long get() { - long normalized = (_incrementingId.getAndIncrement() & Long.MAX_VALUE) % OFFSET; + long normalized = ((_incrementingId.getAndIncrement() & Long.MAX_VALUE) % (OFFSET / 10)) * 10; return _mask + normalized; } } diff --git a/pinot-broker/src/test/java/org/apache/pinot/broker/requesthandler/BrokerRequestIdGeneratorTest.java b/pinot-broker/src/test/java/org/apache/pinot/broker/requesthandler/BrokerRequestIdGeneratorTest.java new file mode 100644 index 00000000000..dab39643832 --- /dev/null +++ b/pinot-broker/src/test/java/org/apache/pinot/broker/requesthandler/BrokerRequestIdGeneratorTest.java @@ -0,0 +1,21 @@ +package org.apache.pinot.broker.requesthandler; + +import org.testng.annotations.Test; + +import static org.testng.Assert.*; + + +public class BrokerRequestIdGeneratorTest { + + @Test + public void testGet() { + BrokerRequestIdGenerator gen = new BrokerRequestIdGenerator("foo"); + long id = gen.get(); + assertEquals(id % 10, 0); + assertEquals(id / 10 % 10, 0); + + id = gen.get(); + assertEquals(id % 10, 0); + assertEquals(id / 10 % 10, 1); + } +} \ No newline at end of file From 5ed97ace2c08f4b45b11dcbb4334f24efbe47f45 Mon Sep 17 00:00:00 2001 From: egalpin Date: Tue, 8 Oct 2024 15:01:09 -0700 Subject: [PATCH 29/49] WIP - Use tableName not query hash --- .../query/reduce/BrokerReduceService.java | 2 +- .../core/transport/AsyncQueryResponse.java | 41 +++++++++++-------- .../pinot/core/transport/QueryResponse.java | 15 +------ .../pinot/core/transport/QueryRouter.java | 7 +++- 4 files changed, 32 insertions(+), 33 deletions(-) diff --git a/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/BrokerReduceService.java b/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/BrokerReduceService.java index d0f195142b8..827e95bdf98 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/BrokerReduceService.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/BrokerReduceService.java @@ -123,7 +123,7 @@ public BrokerResponseNative reduceOnDataTable(BrokerRequest brokerRequest, Broke } } - String tableName = serverBrokerRequest.getQuerySource().getTableName(); + String tableName = serverBrokerRequest.getPinotQuery().getDataSource().getTableName(); String rawTableName = TableNameBuilder.extractRawTableName(tableName); // Set execution statistics and Update broker metrics. diff --git a/pinot-core/src/main/java/org/apache/pinot/core/transport/AsyncQueryResponse.java b/pinot-core/src/main/java/org/apache/pinot/core/transport/AsyncQueryResponse.java index 6b0b27a0895..de79967c1b1 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/transport/AsyncQueryResponse.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/transport/AsyncQueryResponse.java @@ -103,22 +103,12 @@ private Map> getFlatResponses() { } @Override - public Map getCurrentResponses() { + public Map> getCurrentResponses() { return getFlatResponses(); } @Override - public Map> getCurrentResponsesPerServer() { - return getFlatResponses(); - } - - @Override - public Map getFinalResponses() { - - } - - @Override - public Map> getFinalResponsesPerServer() + public Map> getFinalResponses() throws InterruptedException { Map> flatResponses = getFlatResponses(); try { @@ -159,7 +149,7 @@ public String getServerStats() { @Override public long getServerResponseDelayMs(ServerRoutingInstance serverRoutingInstance) { - // TODO(egalpin): How to get query hash here? + // TODO(egalpin): How to get tableName here? return -1L; // return _responseMap.get(serverRoutingInstance).getResponseDelayMs(); } @@ -186,20 +176,35 @@ public long getTimeoutMs() { } void markRequestSubmitted(ServerRoutingInstance serverRoutingInstance, InstanceRequest instanceRequest) { - _responses.get(serverRoutingInstance).get(instanceRequest.getQuery().getPinotQuery().hashCode()) + _responses.get(serverRoutingInstance).get(instanceRequest.getQuery().getPinotQuery().getDataSource().getTableName()) .markRequestSubmitted(); } void markRequestSent(ServerRoutingInstance serverRoutingInstance, InstanceRequest instanceRequest, int requestSentLatencyMs) { - _responses.get(serverRoutingInstance).get(instanceRequest.getQuery().getPinotQuery().hashCode()) + _responses.get(serverRoutingInstance).get(instanceRequest.getQuery().getPinotQuery().getDataSource().getTableName()) .markRequestSent(requestSentLatencyMs); } - void receiveDataTable(ServerRoutingInstance serverRoutingInstance, String tableName, DataTable dataTable, + void receiveDataTable(ServerRoutingInstance serverRoutingInstance, DataTable dataTable, int responseSize, int deserializationTimeMs) { - ServerResponse response = _responses.get(serverRoutingInstance).get(tableName); - response.receiveDataTable(dataTable, responseSize, deserializationTimeMs); + String tableName = dataTable.getMetadata().get(DataTable.MetadataKey.TABLE.getName()); + // tableName can be null but only in the case of version rollout. tableName will only be null when servers are not + // yet running the version where tableName is included in DataTable metadata. For the time being, it is safe to + // assume that a given server will have only been sent 1 or 2 requests (REALTIME, OFFLINE, or both). We can simply + // iterate through responses associated with a server and make use of the first response with null data table. + // TODO(egalpin): The null handling for tableName can and should be removed in a later release. + if (tableName == null) { + for (ServerResponse serverResponse : _responses.get(serverRoutingInstance).values()) { + if (serverResponse.getDataTable() == null) { + serverResponse.receiveDataTable(dataTable, responseSize, deserializationTimeMs); + break; + } + } + } else { + _responses.get(serverRoutingInstance).get(tableName).receiveDataTable(dataTable, responseSize, deserializationTimeMs); + } + _countDownLatch.countDown(); _numServersResponded.getAndIncrement(); } diff --git a/pinot-core/src/main/java/org/apache/pinot/core/transport/QueryResponse.java b/pinot-core/src/main/java/org/apache/pinot/core/transport/QueryResponse.java index 72260ade157..caa91a3ff75 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/transport/QueryResponse.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/transport/QueryResponse.java @@ -49,24 +49,13 @@ enum Status { /** * Returns the current server responses without blocking. - * - * @deprecated use {@link #getCurrentResponsesPerServer()} instead. */ - @Deprecated(forRemoval = true) - Map getCurrentResponses(); - - Map> getCurrentResponsesPerServer(); + Map> getCurrentResponses(); /** * Waits until the query is done (COMPLETED, FAILED or TIMED_OUT) and returns the final server responses. - * - * @deprecated use {@link #getFinalResponsesPerServer()} instead. */ - @Deprecated(forRemoval = true) - Map getFinalResponses() - throws InterruptedException; - - Map> getFinalResponsesPerServer() + Map> getFinalResponses() throws InterruptedException; /** diff --git a/pinot-core/src/main/java/org/apache/pinot/core/transport/QueryRouter.java b/pinot-core/src/main/java/org/apache/pinot/core/transport/QueryRouter.java index 6f135b4e827..8829661f5dc 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/transport/QueryRouter.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/transport/QueryRouter.java @@ -102,6 +102,10 @@ public AsyncQueryResponse submitQuery(long requestId, String rawTableName, for (Map.Entry> entry : queryRoutingTable.entrySet()) { for (ServerQueryRoutingContext serverQueryRoutingContext : entry.getValue()) { ServerRoutingInstance serverRoutingInstance = entry.getKey().toServerRoutingInstance(preferTls); + + // TODO(egalpin): inject value into request ID to indicate REALTIME Vs OFFLINE table type? +// serverQueryRoutingContext.getTableType(); + InstanceRequest instanceRequest = getInstanceRequest(requestId, serverQueryRoutingContext.getBrokerRequest(), serverQueryRoutingContext.getSegmentsToQuery()); if (!skipUnavailableServers && QueryOptionsUtils.isSkipUnavailableServers( @@ -180,6 +184,7 @@ public void shutDown() { void receiveDataTable(ServerRoutingInstance serverRoutingInstance, DataTable dataTable, int responseSize, int deserializationTimeMs) { long requestId = Long.parseLong(dataTable.getMetadata().get(MetadataKey.REQUEST_ID.getName())); + // TODO(egalpin): parse out table type based on last digit of request ID? // TODO(egalpin): How can we handle rolling out brokers expecting query_hash to be present, but while old server // versions are still running and not yet setting the query hash from their side? If possible, deploying servers @@ -189,7 +194,7 @@ void receiveDataTable(ServerRoutingInstance serverRoutingInstance, DataTable dat // Query future might be null if the query is already done (maybe due to failure) if (asyncQueryResponse != null) { - asyncQueryResponse.receiveDataTable(serverRoutingInstance, tableName, dataTable, responseSize, + asyncQueryResponse.receiveDataTable(serverRoutingInstance, dataTable, responseSize, deserializationTimeMs); } } From 4941c0b89c7dc6cdb86963cf4664e57d398ee511 Mon Sep 17 00:00:00 2001 From: egalpin Date: Tue, 8 Oct 2024 16:34:35 -0700 Subject: [PATCH 30/49] Use requestId to infer tableType if null --- .../BrokerRequestIdGenerator.java | 4 +- .../core/transport/AsyncQueryResponse.java | 66 +++++++++++++------ .../pinot/core/transport/QueryRouter.java | 28 +++++--- 3 files changed, 67 insertions(+), 31 deletions(-) diff --git a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BrokerRequestIdGenerator.java b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BrokerRequestIdGenerator.java index aee53c180b2..74e47ece763 100644 --- a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BrokerRequestIdGenerator.java +++ b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BrokerRequestIdGenerator.java @@ -19,6 +19,8 @@ package org.apache.pinot.broker.requesthandler; import java.util.concurrent.atomic.AtomicLong; +import org.apache.pinot.common.Constants; + /** * An ID generator to produce a global unique identifier for each query, used in v1/v2 engine for tracking and @@ -43,7 +45,7 @@ public BrokerRequestIdGenerator(String brokerId) { } public long get() { - long normalized = ((_incrementingId.getAndIncrement() & Long.MAX_VALUE) % (OFFSET / 10)) * 10; + long normalized = ((_incrementingId.getAndIncrement() & Long.MAX_VALUE) % (OFFSET / Constants.TABLE_TYPE_OFFSET)) * Constants.TABLE_TYPE_OFFSET; return _mask + normalized; } } diff --git a/pinot-core/src/main/java/org/apache/pinot/core/transport/AsyncQueryResponse.java b/pinot-core/src/main/java/org/apache/pinot/core/transport/AsyncQueryResponse.java index de79967c1b1..ac9f70c837f 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/transport/AsyncQueryResponse.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/transport/AsyncQueryResponse.java @@ -29,9 +29,12 @@ import java.util.concurrent.atomic.AtomicReference; import javax.annotation.Nullable; import javax.annotation.concurrent.ThreadSafe; +import org.apache.pinot.common.Constants; import org.apache.pinot.common.datatable.DataTable; import org.apache.pinot.common.request.InstanceRequest; import org.apache.pinot.core.transport.server.routing.stats.ServerRoutingStatsManager; +import org.apache.pinot.spi.config.table.TableType; +import org.apache.pinot.spi.utils.builder.TableNameBuilder; /** @@ -40,7 +43,10 @@ @ThreadSafe public class AsyncQueryResponse implements QueryResponse { private final QueryRouter _queryRouter; - private final long _requestId; + + // TODO(egalpin): Remove the concept of truncated request ID in next major release after all servers are expected + // to send back tableName in DataTable metadata + private final long _truncatedRequestId; private final AtomicReference _status = new AtomicReference<>(Status.IN_PROGRESS); private final AtomicInteger _numServersResponded = new AtomicInteger(); private final ConcurrentHashMap> _responses; @@ -52,11 +58,11 @@ public class AsyncQueryResponse implements QueryResponse { private volatile ServerRoutingInstance _failedServer; private volatile Exception _exception; - public AsyncQueryResponse(QueryRouter queryRouter, long requestId, + public AsyncQueryResponse(QueryRouter queryRouter, long truncatedRequestId, Map> requestMap, long startTimeMs, long timeoutMs, ServerRoutingStatsManager serverRoutingStatsManager) { _queryRouter = queryRouter; - _requestId = requestId; + _truncatedRequestId = truncatedRequestId; _responses = new ConcurrentHashMap<>(); _serverRoutingStatsManager = serverRoutingStatsManager; int numQueriesIssued = 0; @@ -65,7 +71,8 @@ public AsyncQueryResponse(QueryRouter queryRouter, long requestId, // Record stats related to query submission just before sending the request. Otherwise, if the response is // received immediately, there's a possibility of updating query response stats before updating query // submission stats. - _serverRoutingStatsManager.recordStatsForQuerySubmission(requestId, serverRequests.getKey().getInstanceId()); + _serverRoutingStatsManager.recordStatsForQuerySubmission(_truncatedRequestId, + serverRequests.getKey().getInstanceId()); _responses.computeIfAbsent(serverRequests.getKey(), k -> new ConcurrentHashMap<>()) .put(request.getQuery().getPinotQuery().getDataSource().getTableName(), new ServerResponse(startTimeMs)); @@ -124,11 +131,11 @@ public Map> getFinalResponses() // from the server. Hence we set the latency to the timeout value. long latency = (response != null && response.getResponseDelayMs() >= 0) ? response.getResponseDelayMs() : _timeoutMs; - _serverRoutingStatsManager.recordStatsUponResponseArrival(_requestId, + _serverRoutingStatsManager.recordStatsUponResponseArrival(_truncatedRequestId, serverResponses.getKey().getInstanceId(), latency); } } - _queryRouter.markQueryDone(_requestId); + _queryRouter.markQueryDone(_truncatedRequestId); } } @@ -149,9 +156,13 @@ public String getServerStats() { @Override public long getServerResponseDelayMs(ServerRoutingInstance serverRoutingInstance) { - // TODO(egalpin): How to get tableName here? - return -1L; -// return _responseMap.get(serverRoutingInstance).getResponseDelayMs(); + int count = 0; + long sum = 0; + for (ServerResponse serverResponse : _responses.get(serverRoutingInstance).values()) { + count += 1; + sum += serverResponse.getResponseDelayMs(); + } + return sum / count; } @Nullable @@ -167,7 +178,7 @@ public Exception getException() { @Override public long getRequestId() { - return _requestId; + return _truncatedRequestId; } @Override @@ -186,8 +197,27 @@ void markRequestSent(ServerRoutingInstance serverRoutingInstance, InstanceReques .markRequestSent(requestSentLatencyMs); } - void receiveDataTable(ServerRoutingInstance serverRoutingInstance, DataTable dataTable, - int responseSize, int deserializationTimeMs) { + @Nullable + private String inferTableName(ServerRoutingInstance serverRoutingInstance, DataTable dataTable) { + TableType tableType; + long requestId = Long.parseLong(dataTable.getMetadata().get(DataTable.MetadataKey.REQUEST_ID.getName())); + if ((requestId & 0x1) == Constants.REALTIME_TABLE_DIGIT) { + tableType = TableType.REALTIME; + } else { + tableType = TableType.OFFLINE; + } + + for (Map.Entry serverResponseEntry : _responses.get(serverRoutingInstance).entrySet()) { + String tableName = serverResponseEntry.getKey(); + if (!TableNameBuilder.getTableTypeFromTableName(tableName).equals(tableType)) { + return tableName; + } + } + return null; + } + + void receiveDataTable(ServerRoutingInstance serverRoutingInstance, DataTable dataTable, int responseSize, + int deserializationTimeMs) { String tableName = dataTable.getMetadata().get(DataTable.MetadataKey.TABLE.getName()); // tableName can be null but only in the case of version rollout. tableName will only be null when servers are not // yet running the version where tableName is included in DataTable metadata. For the time being, it is safe to @@ -195,16 +225,12 @@ void receiveDataTable(ServerRoutingInstance serverRoutingInstance, DataTable dat // iterate through responses associated with a server and make use of the first response with null data table. // TODO(egalpin): The null handling for tableName can and should be removed in a later release. if (tableName == null) { - for (ServerResponse serverResponse : _responses.get(serverRoutingInstance).values()) { - if (serverResponse.getDataTable() == null) { - serverResponse.receiveDataTable(dataTable, responseSize, deserializationTimeMs); - break; - } - } - } else { - _responses.get(serverRoutingInstance).get(tableName).receiveDataTable(dataTable, responseSize, deserializationTimeMs); + tableName = inferTableName(serverRoutingInstance, dataTable); } + _responses.get(serverRoutingInstance).get(tableName) + .receiveDataTable(dataTable, responseSize, deserializationTimeMs); + _countDownLatch.countDown(); _numServersResponded.getAndIncrement(); } diff --git a/pinot-core/src/main/java/org/apache/pinot/core/transport/QueryRouter.java b/pinot-core/src/main/java/org/apache/pinot/core/transport/QueryRouter.java index 8829661f5dc..f660d5ffe86 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/transport/QueryRouter.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/transport/QueryRouter.java @@ -28,6 +28,7 @@ import javax.annotation.concurrent.ThreadSafe; import org.apache.commons.collections4.CollectionUtils; import org.apache.commons.lang3.tuple.Pair; +import org.apache.pinot.common.Constants; import org.apache.pinot.common.config.NettyConfig; import org.apache.pinot.common.config.TlsConfig; import org.apache.pinot.common.datatable.DataTable; @@ -38,6 +39,7 @@ import org.apache.pinot.common.request.InstanceRequest; import org.apache.pinot.common.utils.config.QueryOptionsUtils; import org.apache.pinot.core.transport.server.routing.stats.ServerRoutingStatsManager; +import org.apache.pinot.spi.config.table.TableType; import org.apache.pinot.spi.utils.CommonConstants; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -104,10 +106,13 @@ public AsyncQueryResponse submitQuery(long requestId, String rawTableName, ServerRoutingInstance serverRoutingInstance = entry.getKey().toServerRoutingInstance(preferTls); // TODO(egalpin): inject value into request ID to indicate REALTIME Vs OFFLINE table type? -// serverQueryRoutingContext.getTableType(); + long tableTypeDigit = + serverQueryRoutingContext.getTableType().equals(TableType.OFFLINE) ? Constants.OFFLINE_TABLE_DIGIT + : Constants.REALTIME_TABLE_DIGIT; - InstanceRequest instanceRequest = getInstanceRequest(requestId, serverQueryRoutingContext.getBrokerRequest(), - serverQueryRoutingContext.getSegmentsToQuery()); + InstanceRequest instanceRequest = + getInstanceRequest(requestId + tableTypeDigit, serverQueryRoutingContext.getBrokerRequest(), + serverQueryRoutingContext.getSegmentsToQuery()); if (!skipUnavailableServers && QueryOptionsUtils.isSkipUnavailableServers( serverQueryRoutingContext.getBrokerRequest().getPinotQuery().getQueryOptions())) { // Any single query having this option set will set it for all queries. This option should not be possible @@ -120,10 +125,15 @@ public AsyncQueryResponse submitQuery(long requestId, String rawTableName, } // Create the asynchronous query response with the request map + long truncatedRequestId = requestId / Constants.TABLE_TYPE_OFFSET; + + // Map the request using all but the last digit. This way, servers will return request IDs where the last digit + // (injected above) will indicate table type, but both will be able to be divided by the TABLE_TYPE_OFFSET in + // order to properly map back to this originating request AsyncQueryResponse asyncQueryResponse = - new AsyncQueryResponse(this, requestId, requestMap, System.currentTimeMillis(), timeoutMs, + new AsyncQueryResponse(this, truncatedRequestId, requestMap, System.currentTimeMillis(), timeoutMs, _serverRoutingStatsManager); - _asyncQueryResponseMap.put(requestId, asyncQueryResponse); + _asyncQueryResponseMap.put(truncatedRequestId, asyncQueryResponse); for (Map.Entry> serverRequests : requestMap.entrySet()) { for (InstanceRequest request : serverRequests.getValue()) { @@ -136,14 +146,14 @@ public AsyncQueryResponse submitQuery(long requestId, String rawTableName, if (ServerChannels.CHANNEL_LOCK_TIMEOUT_MSG.equals(e.getMessage())) { _brokerMetrics.addMeteredTableValue(rawTableName, BrokerMeter.REQUEST_CHANNEL_LOCK_TIMEOUT_EXCEPTIONS, 1); } - markQueryFailed(requestId, serverRoutingInstance, asyncQueryResponse, e); + markQueryFailed(truncatedRequestId, serverRoutingInstance, asyncQueryResponse, e); break; } catch (Exception e) { _brokerMetrics.addMeteredTableValue(rawTableName, BrokerMeter.REQUEST_SEND_EXCEPTIONS, 1); if (skipUnavailableServers) { asyncQueryResponse.skipServerResponse(); } else { - markQueryFailed(requestId, serverRoutingInstance, asyncQueryResponse, e); + markQueryFailed(truncatedRequestId, serverRoutingInstance, asyncQueryResponse, e); break; } } @@ -189,13 +199,11 @@ void receiveDataTable(ServerRoutingInstance serverRoutingInstance, DataTable dat // TODO(egalpin): How can we handle rolling out brokers expecting query_hash to be present, but while old server // versions are still running and not yet setting the query hash from their side? If possible, deploying servers // first would work. - String tableName = dataTable.getMetadata().get(MetadataKey.TABLE.getName()); AsyncQueryResponse asyncQueryResponse = _asyncQueryResponseMap.get(requestId); // Query future might be null if the query is already done (maybe due to failure) if (asyncQueryResponse != null) { - asyncQueryResponse.receiveDataTable(serverRoutingInstance, dataTable, responseSize, - deserializationTimeMs); + asyncQueryResponse.receiveDataTable(serverRoutingInstance, dataTable, responseSize, deserializationTimeMs); } } From 0cd47ca689c39d12a44deb0cb6fdb510cc8ed9b3 Mon Sep 17 00:00:00 2001 From: egalpin Date: Tue, 8 Oct 2024 16:48:51 -0700 Subject: [PATCH 31/49] Fixes post-merge import --- .../java/org/apache/pinot/core/transport/AsyncQueryResponse.java | 1 + 1 file changed, 1 insertion(+) diff --git a/pinot-core/src/main/java/org/apache/pinot/core/transport/AsyncQueryResponse.java b/pinot-core/src/main/java/org/apache/pinot/core/transport/AsyncQueryResponse.java index b6ddf5e77f6..9cf684dbc0a 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/transport/AsyncQueryResponse.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/transport/AsyncQueryResponse.java @@ -35,6 +35,7 @@ import org.apache.pinot.common.exception.QueryException; import org.apache.pinot.spi.config.table.TableType; import org.apache.pinot.spi.utils.builder.TableNameBuilder; +import org.apache.pinot.core.transport.server.routing.stats.ServerRoutingStatsManager; /** From c280cb81117df5e2a3ab4dab9b7480efe5b9fe6d Mon Sep 17 00:00:00 2001 From: egalpin Date: Tue, 8 Oct 2024 16:49:13 -0700 Subject: [PATCH 32/49] Adds shared constants for request ID hacking --- .../org/apache/pinot/common/Constants.java | 26 +++++++++++++++++++ 1 file changed, 26 insertions(+) create mode 100644 pinot-common/src/main/java/org/apache/pinot/common/Constants.java diff --git a/pinot-common/src/main/java/org/apache/pinot/common/Constants.java b/pinot-common/src/main/java/org/apache/pinot/common/Constants.java new file mode 100644 index 00000000000..12e5b4dfa7d --- /dev/null +++ b/pinot-common/src/main/java/org/apache/pinot/common/Constants.java @@ -0,0 +1,26 @@ +/** + * 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.common; + +public class Constants { + public static final int TABLE_TYPE_OFFSET = 10; + public static final int OFFLINE_TABLE_DIGIT = 0; + public static final int REALTIME_TABLE_DIGIT = 1; + private Constants() {} +} From c5837bde3630ec4092e282d8385c06034c13e841 Mon Sep 17 00:00:00 2001 From: egalpin Date: Tue, 8 Oct 2024 16:55:37 -0700 Subject: [PATCH 33/49] Linting fixes --- .../BrokerRequestIdGenerator.java | 3 ++- .../BrokerRequestIdGeneratorTest.java | 20 ++++++++++++++++++- .../org/apache/pinot/common/Constants.java | 3 ++- .../core/transport/AsyncQueryResponse.java | 8 ++++---- 4 files changed, 27 insertions(+), 7 deletions(-) diff --git a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BrokerRequestIdGenerator.java b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BrokerRequestIdGenerator.java index 74e47ece763..d8849daec08 100644 --- a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BrokerRequestIdGenerator.java +++ b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BrokerRequestIdGenerator.java @@ -45,7 +45,8 @@ public BrokerRequestIdGenerator(String brokerId) { } public long get() { - long normalized = ((_incrementingId.getAndIncrement() & Long.MAX_VALUE) % (OFFSET / Constants.TABLE_TYPE_OFFSET)) * Constants.TABLE_TYPE_OFFSET; + long normalized = ((_incrementingId.getAndIncrement() & Long.MAX_VALUE) % (OFFSET / Constants.TABLE_TYPE_OFFSET)) + * Constants.TABLE_TYPE_OFFSET; return _mask + normalized; } } diff --git a/pinot-broker/src/test/java/org/apache/pinot/broker/requesthandler/BrokerRequestIdGeneratorTest.java b/pinot-broker/src/test/java/org/apache/pinot/broker/requesthandler/BrokerRequestIdGeneratorTest.java index dab39643832..ee6efcaa2e9 100644 --- a/pinot-broker/src/test/java/org/apache/pinot/broker/requesthandler/BrokerRequestIdGeneratorTest.java +++ b/pinot-broker/src/test/java/org/apache/pinot/broker/requesthandler/BrokerRequestIdGeneratorTest.java @@ -1,3 +1,21 @@ +/** + * 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.broker.requesthandler; import org.testng.annotations.Test; @@ -18,4 +36,4 @@ public void testGet() { assertEquals(id % 10, 0); assertEquals(id / 10 % 10, 1); } -} \ No newline at end of file +} diff --git a/pinot-common/src/main/java/org/apache/pinot/common/Constants.java b/pinot-common/src/main/java/org/apache/pinot/common/Constants.java index 12e5b4dfa7d..38bc0b7e087 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/Constants.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/Constants.java @@ -22,5 +22,6 @@ public class Constants { public static final int TABLE_TYPE_OFFSET = 10; public static final int OFFLINE_TABLE_DIGIT = 0; public static final int REALTIME_TABLE_DIGIT = 1; - private Constants() {} + private Constants() { + } } diff --git a/pinot-core/src/main/java/org/apache/pinot/core/transport/AsyncQueryResponse.java b/pinot-core/src/main/java/org/apache/pinot/core/transport/AsyncQueryResponse.java index 9cf684dbc0a..b1d42266eb4 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/transport/AsyncQueryResponse.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/transport/AsyncQueryResponse.java @@ -98,8 +98,8 @@ public int getNumServersResponded() { private Map> getFlatResponses() { Map> flattened = new HashMap<>(); - for (Map.Entry> serverResponses : - _responses.entrySet()) { + for (Map.Entry> serverResponses + : _responses.entrySet()) { ServerRoutingInstance serverRoutingInstance = serverResponses.getKey(); for (ServerResponse serverResponse : serverResponses.getValue().values()) { @@ -171,8 +171,8 @@ private boolean hasServerNotResponded(ServerResponse response) { public String getServerStats() { StringBuilder stringBuilder = new StringBuilder( "(Server=SubmitDelayMs,ResponseDelayMs,ResponseSize,DeserializationTimeMs,RequestSentDelayMs)"); - for (Map.Entry> serverResponses : - _responses.entrySet()) { + for (Map.Entry> serverResponses + : _responses.entrySet()) { for (Map.Entry responsePair : serverResponses.getValue().entrySet()) { ServerRoutingInstance serverRoutingInstance = serverResponses.getKey(); stringBuilder.append(';').append(serverRoutingInstance.getShortName()).append('=') From 598553ed63eeb179b954865760034e259f9886a9 Mon Sep 17 00:00:00 2001 From: egalpin Date: Tue, 8 Oct 2024 17:04:18 -0700 Subject: [PATCH 34/49] Adds hacky access to table type inference from request ID --- .../core/query/reduce/ExecutionStatsAggregator.java | 3 +++ .../apache/pinot/core/transport/AsyncQueryResponse.java | 9 +++++++-- 2 files changed, 10 insertions(+), 2 deletions(-) diff --git a/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/ExecutionStatsAggregator.java b/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/ExecutionStatsAggregator.java index 18f7739e407..f946d60afce 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/ExecutionStatsAggregator.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/ExecutionStatsAggregator.java @@ -30,6 +30,7 @@ import org.apache.pinot.common.metrics.BrokerTimer; import org.apache.pinot.common.response.broker.BrokerResponseNative; import org.apache.pinot.common.response.broker.QueryProcessingException; +import org.apache.pinot.core.transport.AsyncQueryResponse; import org.apache.pinot.core.transport.ServerRoutingInstance; import org.apache.pinot.spi.config.table.TableType; import org.apache.pinot.spi.utils.builder.TableNameBuilder; @@ -77,6 +78,8 @@ public void aggregate(ServerRoutingInstance serverRoutingInstance, DataTable dat TableType tableType = null; if (metadata.get(DataTable.MetadataKey.TABLE.getName()) != null) { tableType = TableNameBuilder.getTableTypeFromTableName(metadata.get(DataTable.MetadataKey.TABLE.getName())); + } else if (metadata.get(DataTable.MetadataKey.REQUEST_ID.getName()) != null) { + tableType = AsyncQueryResponse.inferTableType(dataTable); } // Reduce on trace info. diff --git a/pinot-core/src/main/java/org/apache/pinot/core/transport/AsyncQueryResponse.java b/pinot-core/src/main/java/org/apache/pinot/core/transport/AsyncQueryResponse.java index b1d42266eb4..d9754afe453 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/transport/AsyncQueryResponse.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/transport/AsyncQueryResponse.java @@ -225,8 +225,7 @@ void markRequestSent(ServerRoutingInstance serverRoutingInstance, InstanceReques .markRequestSent(requestSentLatencyMs); } - @Nullable - private String inferTableName(ServerRoutingInstance serverRoutingInstance, DataTable dataTable) { + public static TableType inferTableType(DataTable dataTable) { TableType tableType; long requestId = Long.parseLong(dataTable.getMetadata().get(DataTable.MetadataKey.REQUEST_ID.getName())); if ((requestId & 0x1) == Constants.REALTIME_TABLE_DIGIT) { @@ -234,6 +233,12 @@ private String inferTableName(ServerRoutingInstance serverRoutingInstance, DataT } else { tableType = TableType.OFFLINE; } + return tableType; + } + + @Nullable + private String inferTableName(ServerRoutingInstance serverRoutingInstance, DataTable dataTable) { + TableType tableType = inferTableType(dataTable); for (Map.Entry serverResponseEntry : _responses.get(serverRoutingInstance).entrySet()) { String tableName = serverResponseEntry.getKey(); From f7fc1feb3b1e6c2b803b5180dd160c015fe8f97b Mon Sep 17 00:00:00 2001 From: egalpin Date: Thu, 10 Oct 2024 15:10:24 -0700 Subject: [PATCH 35/49] Removes QUERY_HASH from DataTable metadata --- .../pinot/common/datatable/DataTable.java | 5 +- .../query/request/ServerQueryRequest.java | 5 +- .../core/transport/AsyncQueryResponse.java | 6 +- .../core/transport/DataTableHandler.java | 10 +-- .../pinot/core/transport/QueryRouter.java | 2 +- .../core/transport/QueryRoutingTest.java | 88 +++++++++++-------- 6 files changed, 64 insertions(+), 52 deletions(-) diff --git a/pinot-common/src/main/java/org/apache/pinot/common/datatable/DataTable.java b/pinot-common/src/main/java/org/apache/pinot/common/datatable/DataTable.java index e3610d35a0c..e11b632c169 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/datatable/DataTable.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/datatable/DataTable.java @@ -139,12 +139,11 @@ enum MetadataKey { OPERATOR_ID(31, "operatorId", MetadataValueType.STRING), OPERATOR_EXEC_START_TIME_MS(32, "operatorExecStartTimeMs", MetadataValueType.LONG), OPERATOR_EXEC_END_TIME_MS(33, "operatorExecEndTimeMs", MetadataValueType.LONG), - MAX_ROWS_IN_JOIN_REACHED(34, "maxRowsInJoinReached", MetadataValueType.STRING), - QUERY_HASH(35, "queryHash", MetadataValueType.INT); + MAX_ROWS_IN_JOIN_REACHED(34, "maxRowsInJoinReached", MetadataValueType.STRING); // We keep this constant to track the max id added so far for backward compatibility. // Increase it when adding new keys, but NEVER DECREASE IT!!! - private static final int MAX_ID = 35; + private static final int MAX_ID = 34; private static final MetadataKey[] ID_TO_ENUM_KEY_MAP = new MetadataKey[MAX_ID + 1]; private static final Map NAME_TO_ENUM_KEY_MAP = new HashMap<>(); diff --git a/pinot-core/src/main/java/org/apache/pinot/core/query/request/ServerQueryRequest.java b/pinot-core/src/main/java/org/apache/pinot/core/query/request/ServerQueryRequest.java index a14cd9a7fb5..465397838a9 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/query/request/ServerQueryRequest.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/query/request/ServerQueryRequest.java @@ -73,7 +73,7 @@ public ServerQueryRequest(InstanceRequest instanceRequest, ServerMetrics serverM _optionalSegments = instanceRequest.getOptionalSegments(); _queryContext = getQueryContext(instanceRequest.getQuery().getPinotQuery()); // Method to set table name needs to match whats in AsyncQueryResponse - _tableName = instanceRequest.getQuery().getPinotQuery().getDataSource().getTableName(); + _tableName = _queryContext.getTableName(); _queryId = QueryIdUtils.getQueryId(_brokerId, _requestId, TableNameBuilder.getTableTypeFromTableName(_queryContext.getTableName())); _timerContext = new TimerContext(_queryContext.getTableName(), serverMetrics, queryArrivalTimeMs); @@ -105,7 +105,7 @@ public ServerQueryRequest(Server.ServerRequest serverRequest, ServerMetrics serv throw new UnsupportedOperationException("Unsupported payloadType: " + payloadType); } _queryContext = getQueryContext(brokerRequest.getPinotQuery()); - _tableName = brokerRequest.getPinotQuery().getDataSource().getTableName(); + _tableName = _queryContext.getTableName(); _queryId = QueryIdUtils.getQueryId(_brokerId, _requestId, TableNameBuilder.getTableTypeFromTableName(_queryContext.getTableName())); _timerContext = new TimerContext(_queryContext.getTableName(), serverMetrics, queryArrivalTimeMs); @@ -127,6 +127,7 @@ public ServerQueryRequest(QueryContext queryContext, List segmentsToQuer _queryId = QueryIdUtils.getQueryId(_brokerId, _requestId, TableNameBuilder.getTableTypeFromTableName(_queryContext.getTableName())); + _tableName = _queryContext.getTableName(); _segmentsToQuery = segmentsToQuery; _optionalSegments = null; diff --git a/pinot-core/src/main/java/org/apache/pinot/core/transport/AsyncQueryResponse.java b/pinot-core/src/main/java/org/apache/pinot/core/transport/AsyncQueryResponse.java index d9754afe453..f9f3bfd4dff 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/transport/AsyncQueryResponse.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/transport/AsyncQueryResponse.java @@ -31,11 +31,11 @@ import javax.annotation.concurrent.ThreadSafe; import org.apache.pinot.common.Constants; import org.apache.pinot.common.datatable.DataTable; -import org.apache.pinot.common.request.InstanceRequest; import org.apache.pinot.common.exception.QueryException; +import org.apache.pinot.common.request.InstanceRequest; +import org.apache.pinot.core.transport.server.routing.stats.ServerRoutingStatsManager; import org.apache.pinot.spi.config.table.TableType; import org.apache.pinot.spi.utils.builder.TableNameBuilder; -import org.apache.pinot.core.transport.server.routing.stats.ServerRoutingStatsManager; /** @@ -242,7 +242,7 @@ private String inferTableName(ServerRoutingInstance serverRoutingInstance, DataT for (Map.Entry serverResponseEntry : _responses.get(serverRoutingInstance).entrySet()) { String tableName = serverResponseEntry.getKey(); - if (!TableNameBuilder.getTableTypeFromTableName(tableName).equals(tableType)) { + if (TableNameBuilder.getTableTypeFromTableName(tableName).equals(tableType)) { return tableName; } } diff --git a/pinot-core/src/main/java/org/apache/pinot/core/transport/DataTableHandler.java b/pinot-core/src/main/java/org/apache/pinot/core/transport/DataTableHandler.java index 3a117a49912..608dd9844c2 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/transport/DataTableHandler.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/transport/DataTableHandler.java @@ -48,11 +48,6 @@ public DataTableHandler(QueryRouter queryRouter, ServerRoutingInstance serverRou _brokerMetrics = brokerMetrics; } - @Override - public void channelActive(ChannelHandlerContext ctx) { - LOGGER.info("Channel for server: {} is now active", _serverRoutingInstance); - } - @Override public void channelInactive(ChannelHandlerContext ctx) { LOGGER.error("Channel for server: {} is now inactive, marking server down", _serverRoutingInstance); @@ -60,6 +55,11 @@ public void channelInactive(ChannelHandlerContext ctx) { new RuntimeException(String.format("Channel for server: %s is inactive", _serverRoutingInstance))); } + @Override + public void channelActive(ChannelHandlerContext ctx) { + LOGGER.info("Channel for server: {} is now active", _serverRoutingInstance); + } + @Override protected void channelRead0(ChannelHandlerContext ctx, ByteBuf msg) { Tracing.ThreadAccountantOps.setThreadResourceUsageProvider(); diff --git a/pinot-core/src/main/java/org/apache/pinot/core/transport/QueryRouter.java b/pinot-core/src/main/java/org/apache/pinot/core/transport/QueryRouter.java index f660d5ffe86..7793ff25cd1 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/transport/QueryRouter.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/transport/QueryRouter.java @@ -199,7 +199,7 @@ void receiveDataTable(ServerRoutingInstance serverRoutingInstance, DataTable dat // TODO(egalpin): How can we handle rolling out brokers expecting query_hash to be present, but while old server // versions are still running and not yet setting the query hash from their side? If possible, deploying servers // first would work. - AsyncQueryResponse asyncQueryResponse = _asyncQueryResponseMap.get(requestId); + AsyncQueryResponse asyncQueryResponse = _asyncQueryResponseMap.get(requestId / Constants.TABLE_TYPE_OFFSET); // Query future might be null if the query is already done (maybe due to failure) if (asyncQueryResponse != null) { diff --git a/pinot-core/src/test/java/org/apache/pinot/core/transport/QueryRoutingTest.java b/pinot-core/src/test/java/org/apache/pinot/core/transport/QueryRoutingTest.java index e794de34e0c..317c18a5c4d 100644 --- a/pinot-core/src/test/java/org/apache/pinot/core/transport/QueryRoutingTest.java +++ b/pinot-core/src/test/java/org/apache/pinot/core/transport/QueryRoutingTest.java @@ -25,6 +25,7 @@ import java.util.List; import java.util.Map; import org.apache.commons.lang3.tuple.Pair; +import org.apache.pinot.common.Constants; import org.apache.pinot.common.datatable.DataTable; import org.apache.pinot.common.datatable.DataTable.MetadataKey; import org.apache.pinot.common.exception.QueryException; @@ -39,6 +40,7 @@ import org.apache.pinot.core.query.scheduler.QueryScheduler; import org.apache.pinot.core.transport.server.routing.stats.ServerRoutingStatsManager; import org.apache.pinot.server.access.AccessControl; +import org.apache.pinot.spi.config.table.TableType; import org.apache.pinot.spi.env.PinotConfiguration; import org.apache.pinot.spi.utils.CommonConstants; import org.apache.pinot.spi.utils.builder.TableNameBuilder; @@ -138,10 +140,15 @@ private QueryScheduler mockQueryScheduler(int responseDelayMs, DataTable offline QueryScheduler queryScheduler = mock(QueryScheduler.class); when(queryScheduler.submit(any())).thenAnswer(invocation -> { Thread.sleep(responseDelayMs); - String queryHash = String.valueOf(((ServerQueryRequest) invocation.getArguments()[0]).getTableName()); - if (queryHash.equals(realtimeDataTable.getMetadata().get(MetadataKey.QUERY_HASH.getName()))) { + // TODO(egalpin): verify if this arg is tablename + String tableName = String.valueOf(((ServerQueryRequest) invocation.getArguments()[0]).getTableName()); + TableType tableType = TableNameBuilder.getTableTypeFromTableName(tableName); + AsyncQueryResponse.inferTableType(realtimeDataTable); + if (tableName.equals(realtimeDataTable.getMetadata().get(MetadataKey.TABLE.getName())) || tableType.equals( + AsyncQueryResponse.inferTableType(realtimeDataTable))) { return Futures.immediateFuture(realtimeDataTable.toBytes()); - } else if (queryHash.equals(offlineDataTable.getMetadata().get(MetadataKey.QUERY_HASH.getName()))) { + } else if (tableName.equals(offlineDataTable.getMetadata().get(MetadataKey.TABLE.getName())) || tableType.equals( + AsyncQueryResponse.inferTableType(offlineDataTable))) { return Futures.immediateFuture(offlineDataTable.toBytes()); } return Futures.immediateFuture(new byte[0]); @@ -152,17 +159,19 @@ private QueryScheduler mockQueryScheduler(int responseDelayMs, DataTable offline @Test public void testValidResponse() throws Exception { - long requestId = 123; + long requestId = 1230; DataTable offlineDataTable = DataTableBuilderFactory.getEmptyDataTable(); - offlineDataTable.getMetadata().put(MetadataKey.REQUEST_ID.getName(), Long.toString(requestId)); offlineDataTable.getMetadata() - .put(MetadataKey.QUERY_HASH.getName(), Integer.toString(OFFLINE_BROKER_REQUEST.getPinotQuery().hashCode())); + .put(MetadataKey.REQUEST_ID.getName(), Long.toString(requestId + Constants.OFFLINE_TABLE_DIGIT)); + offlineDataTable.getMetadata() + .put(MetadataKey.TABLE.getName(), OFFLINE_BROKER_REQUEST.getQuerySource().getTableName()); byte[] offlineResponseBytes = offlineDataTable.toBytes(); DataTable realtimeDataTable = DataTableBuilderFactory.getEmptyDataTable(); - realtimeDataTable.getMetadata().put(MetadataKey.REQUEST_ID.getName(), Long.toString(requestId)); realtimeDataTable.getMetadata() - .put(MetadataKey.QUERY_HASH.getName(), Integer.toString(REALTIME_BROKER_REQUEST.getPinotQuery().hashCode())); + .put(MetadataKey.REQUEST_ID.getName(), Long.toString(requestId + Constants.REALTIME_TABLE_DIGIT)); + realtimeDataTable.getMetadata() + .put(MetadataKey.TABLE.getName(), REALTIME_BROKER_REQUEST.getQuerySource().getTableName()); byte[] realtimeResponseBytes = realtimeDataTable.toBytes(); String serverId = SERVER_INSTANCE.getInstanceId(); @@ -194,6 +203,7 @@ public void testValidResponse() assertEquals(response.get(SERVER_ROUTING_INSTANCE).size(), 1); serverResponse = response.get(SERVER_ROUTING_INSTANCE).get(0); assertNotNull(serverResponse.getDataTable()); + assertEquals(TableType.REALTIME, AsyncQueryResponse.inferTableType(serverResponse.getDataTable())); assertEquals(serverResponse.getResponseSize(), realtimeResponseBytes.length); _requestCount += 2; waitForStatsUpdate(_requestCount); @@ -209,12 +219,12 @@ public void testValidResponse() int accountedFor = 0; for (ServerResponse serverResponse1 : response.get(SERVER_ROUTING_INSTANCE)) { assertNotNull(serverResponse1.getDataTable()); - if (serverResponse1.getDataTable().getMetadata().get(MetadataKey.QUERY_HASH.getName()) - .equals(offlineDataTable.getMetadata().get(MetadataKey.QUERY_HASH.getName()))) { + if (serverResponse1.getDataTable().getMetadata().get(MetadataKey.TABLE.getName()) + .equals(offlineDataTable.getMetadata().get(MetadataKey.TABLE.getName()))) { assertEquals(serverResponse1.getResponseSize(), offlineResponseBytes.length); accountedFor++; - } else if (serverResponse1.getDataTable().getMetadata().get(MetadataKey.QUERY_HASH.getName()) - .equals(realtimeDataTable.getMetadata().get(MetadataKey.QUERY_HASH.getName()))) { + } else if (serverResponse1.getDataTable().getMetadata().get(MetadataKey.TABLE.getName()) + .equals(realtimeDataTable.getMetadata().get(MetadataKey.TABLE.getName()))) { assertEquals(serverResponse1.getResponseSize(), realtimeResponseBytes.length); accountedFor++; } @@ -279,10 +289,10 @@ public void testLatencyForQueryServerException() // Send a query with ServerSide exception and check if the latency is set to timeout value. Double latencyBefore = _serverRoutingStatsManager.fetchEMALatencyForServer(serverId); AsyncQueryResponse asyncQueryResponse = - _queryRouter.submitQuery(requestId, "testTable", BROKER_REQUEST, ROUTING_TABLE, null, null, 1_000L); - Map response = asyncQueryResponse.getFinalResponses(); + _queryRouter.submitQuery(requestId, "testTable", OFFLINE_ROUTING_TABLE, 1_000L); + Map> response = asyncQueryResponse.getFinalResponses(); assertEquals(response.size(), 1); - assertTrue(response.containsKey(OFFLINE_SERVER_ROUTING_INSTANCE)); + assertTrue(response.containsKey(SERVER_ROUTING_INSTANCE)); _requestCount += 2; waitForStatsUpdate(_requestCount); @@ -308,7 +318,7 @@ public void testLatencyForQueryServerException() @Test public void testLatencyForClientException() throws Exception { - long requestId = 123; + long requestId = 1230; DataTable dataTable = DataTableBuilderFactory.getEmptyDataTable(); dataTable.getMetadata().put(MetadataKey.REQUEST_ID.getName(), Long.toString(requestId)); Exception exception = new UnsupportedOperationException("Caught exception."); @@ -325,11 +335,12 @@ public void testLatencyForClientException() Double latencyBefore = _serverRoutingStatsManager.fetchEMALatencyForServer(serverId); AsyncQueryResponse asyncQueryResponse = - _queryRouter.submitQuery(requestId, "testTable", BROKER_REQUEST, ROUTING_TABLE, null, null, 1_000L); - Map response = asyncQueryResponse.getFinalResponses(); + _queryRouter.submitQuery(requestId, "testTable", OFFLINE_ROUTING_TABLE, 1_000L); + Map> response = asyncQueryResponse.getFinalResponses(); assertEquals(response.size(), 1); - assertTrue(response.containsKey(OFFLINE_SERVER_ROUTING_INSTANCE)); - ServerResponse serverResponse = response.get(OFFLINE_SERVER_ROUTING_INSTANCE); + assertTrue(response.containsKey(SERVER_ROUTING_INSTANCE)); + assertEquals(response.get(SERVER_ROUTING_INSTANCE).size(), 1); + ServerResponse serverResponse = response.get(SERVER_ROUTING_INSTANCE).get(0); _requestCount += 2; waitForStatsUpdate(_requestCount); @@ -352,7 +363,7 @@ public void testLatencyForClientException() @Test public void testLatencyForMultipleExceptions() throws Exception { - long requestId = 123; + long requestId = 1230; DataTable dataTable = DataTableBuilderFactory.getEmptyDataTable(); dataTable.getMetadata().put(MetadataKey.REQUEST_ID.getName(), Long.toString(requestId)); Exception exception = new UnsupportedOperationException("Caught exception."); @@ -372,10 +383,10 @@ public void testLatencyForMultipleExceptions() //server-side exception is seen. Double latencyBefore = _serverRoutingStatsManager.fetchEMALatencyForServer(serverId); AsyncQueryResponse asyncQueryResponse = - _queryRouter.submitQuery(requestId, "testTable", BROKER_REQUEST, ROUTING_TABLE, null, null, 1_000L); - Map response = asyncQueryResponse.getFinalResponses(); + _queryRouter.submitQuery(requestId, "testTable", OFFLINE_ROUTING_TABLE, 1_000L); + Map> response = asyncQueryResponse.getFinalResponses(); assertEquals(response.size(), 1); - assertTrue(response.containsKey(OFFLINE_SERVER_ROUTING_INSTANCE)); + assertTrue(response.containsKey(SERVER_ROUTING_INSTANCE)); _requestCount += 2; waitForStatsUpdate(_requestCount); @@ -400,7 +411,7 @@ public void testLatencyForMultipleExceptions() @Test public void testLatencyForNoException() throws Exception { - long requestId = 123; + long requestId = 1230; DataTable dataTable = DataTableBuilderFactory.getEmptyDataTable(); dataTable.getMetadata().put(MetadataKey.REQUEST_ID.getName(), Long.toString(requestId)); byte[] responseBytes = dataTable.toBytes(); @@ -412,11 +423,12 @@ public void testLatencyForNoException() // Send a valid query and get latency Double latencyBefore = _serverRoutingStatsManager.fetchEMALatencyForServer(serverId); AsyncQueryResponse asyncQueryResponse = - _queryRouter.submitQuery(requestId, "testTable", BROKER_REQUEST, ROUTING_TABLE, null, null, 1_000L); - Map response = asyncQueryResponse.getFinalResponses(); + _queryRouter.submitQuery(requestId, "testTable", OFFLINE_ROUTING_TABLE, 1_000L); + Map> response = asyncQueryResponse.getFinalResponses(); assertEquals(response.size(), 1); - assertTrue(response.containsKey(OFFLINE_SERVER_ROUTING_INSTANCE)); - ServerResponse serverResponse = response.get(OFFLINE_SERVER_ROUTING_INSTANCE); + assertTrue(response.containsKey(SERVER_ROUTING_INSTANCE)); + assertEquals(response.get(SERVER_ROUTING_INSTANCE).size(), 1); + ServerResponse serverResponse = response.get(SERVER_ROUTING_INSTANCE).get(0); _requestCount += 2; waitForStatsUpdate(_requestCount); @@ -437,16 +449,16 @@ public void testLatencyForNoException() @Test public void testNonMatchingRequestId() throws Exception { - long requestId = 123; + long requestId = 1230; DataTable offlineDataTable = DataTableBuilderFactory.getEmptyDataTable(); offlineDataTable.getMetadata().put(MetadataKey.REQUEST_ID.getName(), Long.toString(requestId)); offlineDataTable.getMetadata() - .put(MetadataKey.QUERY_HASH.getName(), Integer.toString(OFFLINE_BROKER_REQUEST.getPinotQuery().hashCode())); + .put(MetadataKey.TABLE.getName(), OFFLINE_BROKER_REQUEST.getQuerySource().getTableName()); DataTable realtimeDataTable = DataTableBuilderFactory.getEmptyDataTable(); realtimeDataTable.getMetadata().put(MetadataKey.REQUEST_ID.getName(), Long.toString(requestId)); realtimeDataTable.getMetadata() - .put(MetadataKey.QUERY_HASH.getName(), Integer.toString(REALTIME_BROKER_REQUEST.getPinotQuery().hashCode())); + .put(MetadataKey.TABLE.getName(), REALTIME_BROKER_REQUEST.getQuerySource().getTableName()); String serverId = SERVER_INSTANCE.getInstanceId(); // Start the server @@ -478,19 +490,19 @@ public void testNonMatchingRequestId() @Test public void testServerDown() throws Exception { - long requestId = 123; + long requestId = 1230; // To avoid flakyness, set timeoutMs to 2000 msec. For some test runs, it can take up to // 1400 msec to mark request as failed. long timeoutMs = 2000L; DataTable offlineDataTable = DataTableBuilderFactory.getEmptyDataTable(); offlineDataTable.getMetadata().put(MetadataKey.REQUEST_ID.getName(), Long.toString(requestId)); offlineDataTable.getMetadata() - .put(MetadataKey.QUERY_HASH.getName(), Integer.toString(OFFLINE_BROKER_REQUEST.getPinotQuery().hashCode())); + .put(MetadataKey.TABLE.getName(), OFFLINE_BROKER_REQUEST.getQuerySource().getTableName()); DataTable realtimeDataTable = DataTableBuilderFactory.getEmptyDataTable(); realtimeDataTable.getMetadata().put(MetadataKey.REQUEST_ID.getName(), Long.toString(requestId)); realtimeDataTable.getMetadata() - .put(MetadataKey.QUERY_HASH.getName(), Integer.toString(REALTIME_BROKER_REQUEST.getPinotQuery().hashCode())); + .put(MetadataKey.TABLE.getName(), REALTIME_BROKER_REQUEST.getQuerySource().getTableName()); String serverId = SERVER_INSTANCE.getInstanceId(); @@ -569,7 +581,7 @@ public void testSkipUnavailableServer() Map.of(serverInstance1, List.of(offlineBrokerRequestContext1), serverInstance2, List.of(offlineBrokerRequestContext2)); - long requestId = 123; + long requestId = 1230; DataSchema dataSchema = new DataSchema(new String[]{"column1"}, new DataSchema.ColumnDataType[]{DataSchema.ColumnDataType.STRING}); DataTableBuilder builder = DataTableBuilderFactory.getDataTableBuilder(dataSchema); @@ -579,7 +591,7 @@ public void testSkipUnavailableServer() DataTable dataTableSuccess = builder.build(); Map dataTableMetadata = dataTableSuccess.getMetadata(); dataTableMetadata.put(MetadataKey.REQUEST_ID.getName(), Long.toString(requestId)); - dataTableMetadata.put(MetadataKey.QUERY_HASH.getName(), Integer.toString(brokerRequest.getPinotQuery().hashCode())); + dataTableMetadata.put(MetadataKey.TABLE.getName(), brokerRequest.getQuerySource().getTableName()); // Only start a single QueryServer, on port from serverInstance1 QueryServer queryServer = getQueryServer(500, dataTableSuccess, dataTableSuccess, port); @@ -611,7 +623,7 @@ public void testSkipUnavailableServer() // Submit the same query without skipUnavailableServers, the servers should not return any response brokerRequest = CalciteSqlCompiler.compileToBrokerRequest("SELECT * FROM testTable_OFFLINE"); - dataTableMetadata.put(MetadataKey.QUERY_HASH.getName(), Integer.toString(brokerRequest.getPinotQuery().hashCode())); + dataTableMetadata.put(MetadataKey.TABLE.getName(), brokerRequest.getQuerySource().getTableName()); offlineBrokerRequestContext1 = new ServerQueryRoutingContext(brokerRequest, Pair.of(Collections.emptyList(), Collections.emptyList()), From 7fb325af61193022b4ad3e5870fa7dc9eceb6ab1 Mon Sep 17 00:00:00 2001 From: egalpin Date: Thu, 10 Oct 2024 16:04:36 -0700 Subject: [PATCH 36/49] Renames Constants to BrokerRequestIdConstants for clarity --- .../requesthandler/BrokerRequestIdGenerator.java | 6 +++--- .../{Constants.java => BrokerRequestIdConstants.java} | 4 ++-- .../pinot/core/transport/AsyncQueryResponse.java | 4 ++-- .../org/apache/pinot/core/transport/QueryRouter.java | 10 +++++----- .../apache/pinot/core/transport/QueryRoutingTest.java | 7 +++---- 5 files changed, 15 insertions(+), 16 deletions(-) rename pinot-common/src/main/java/org/apache/pinot/common/{Constants.java => BrokerRequestIdConstants.java} (92%) diff --git a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BrokerRequestIdGenerator.java b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BrokerRequestIdGenerator.java index d8849daec08..1a2560ecb98 100644 --- a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BrokerRequestIdGenerator.java +++ b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BrokerRequestIdGenerator.java @@ -19,7 +19,7 @@ package org.apache.pinot.broker.requesthandler; import java.util.concurrent.atomic.AtomicLong; -import org.apache.pinot.common.Constants; +import org.apache.pinot.common.BrokerRequestIdConstants; /** @@ -45,8 +45,8 @@ public BrokerRequestIdGenerator(String brokerId) { } public long get() { - long normalized = ((_incrementingId.getAndIncrement() & Long.MAX_VALUE) % (OFFSET / Constants.TABLE_TYPE_OFFSET)) - * Constants.TABLE_TYPE_OFFSET; + long normalized = ((_incrementingId.getAndIncrement() & Long.MAX_VALUE) % (OFFSET / BrokerRequestIdConstants.TABLE_TYPE_OFFSET)) + * BrokerRequestIdConstants.TABLE_TYPE_OFFSET; return _mask + normalized; } } diff --git a/pinot-common/src/main/java/org/apache/pinot/common/Constants.java b/pinot-common/src/main/java/org/apache/pinot/common/BrokerRequestIdConstants.java similarity index 92% rename from pinot-common/src/main/java/org/apache/pinot/common/Constants.java rename to pinot-common/src/main/java/org/apache/pinot/common/BrokerRequestIdConstants.java index 38bc0b7e087..c14940f69e4 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/Constants.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/BrokerRequestIdConstants.java @@ -18,10 +18,10 @@ */ package org.apache.pinot.common; -public class Constants { +public class BrokerRequestIdConstants { public static final int TABLE_TYPE_OFFSET = 10; public static final int OFFLINE_TABLE_DIGIT = 0; public static final int REALTIME_TABLE_DIGIT = 1; - private Constants() { + private BrokerRequestIdConstants() { } } diff --git a/pinot-core/src/main/java/org/apache/pinot/core/transport/AsyncQueryResponse.java b/pinot-core/src/main/java/org/apache/pinot/core/transport/AsyncQueryResponse.java index f9f3bfd4dff..0b2580c62d8 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/transport/AsyncQueryResponse.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/transport/AsyncQueryResponse.java @@ -29,7 +29,7 @@ import java.util.concurrent.atomic.AtomicReference; import javax.annotation.Nullable; import javax.annotation.concurrent.ThreadSafe; -import org.apache.pinot.common.Constants; +import org.apache.pinot.common.BrokerRequestIdConstants; import org.apache.pinot.common.datatable.DataTable; import org.apache.pinot.common.exception.QueryException; import org.apache.pinot.common.request.InstanceRequest; @@ -228,7 +228,7 @@ void markRequestSent(ServerRoutingInstance serverRoutingInstance, InstanceReques public static TableType inferTableType(DataTable dataTable) { TableType tableType; long requestId = Long.parseLong(dataTable.getMetadata().get(DataTable.MetadataKey.REQUEST_ID.getName())); - if ((requestId & 0x1) == Constants.REALTIME_TABLE_DIGIT) { + if ((requestId & 0x1) == BrokerRequestIdConstants.REALTIME_TABLE_DIGIT) { tableType = TableType.REALTIME; } else { tableType = TableType.OFFLINE; diff --git a/pinot-core/src/main/java/org/apache/pinot/core/transport/QueryRouter.java b/pinot-core/src/main/java/org/apache/pinot/core/transport/QueryRouter.java index 7793ff25cd1..1d7174054a1 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/transport/QueryRouter.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/transport/QueryRouter.java @@ -28,7 +28,7 @@ import javax.annotation.concurrent.ThreadSafe; import org.apache.commons.collections4.CollectionUtils; import org.apache.commons.lang3.tuple.Pair; -import org.apache.pinot.common.Constants; +import org.apache.pinot.common.BrokerRequestIdConstants; import org.apache.pinot.common.config.NettyConfig; import org.apache.pinot.common.config.TlsConfig; import org.apache.pinot.common.datatable.DataTable; @@ -107,8 +107,8 @@ public AsyncQueryResponse submitQuery(long requestId, String rawTableName, // TODO(egalpin): inject value into request ID to indicate REALTIME Vs OFFLINE table type? long tableTypeDigit = - serverQueryRoutingContext.getTableType().equals(TableType.OFFLINE) ? Constants.OFFLINE_TABLE_DIGIT - : Constants.REALTIME_TABLE_DIGIT; + serverQueryRoutingContext.getTableType().equals(TableType.OFFLINE) ? BrokerRequestIdConstants.OFFLINE_TABLE_DIGIT + : BrokerRequestIdConstants.REALTIME_TABLE_DIGIT; InstanceRequest instanceRequest = getInstanceRequest(requestId + tableTypeDigit, serverQueryRoutingContext.getBrokerRequest(), @@ -125,7 +125,7 @@ public AsyncQueryResponse submitQuery(long requestId, String rawTableName, } // Create the asynchronous query response with the request map - long truncatedRequestId = requestId / Constants.TABLE_TYPE_OFFSET; + long truncatedRequestId = requestId / BrokerRequestIdConstants.TABLE_TYPE_OFFSET; // Map the request using all but the last digit. This way, servers will return request IDs where the last digit // (injected above) will indicate table type, but both will be able to be divided by the TABLE_TYPE_OFFSET in @@ -199,7 +199,7 @@ void receiveDataTable(ServerRoutingInstance serverRoutingInstance, DataTable dat // TODO(egalpin): How can we handle rolling out brokers expecting query_hash to be present, but while old server // versions are still running and not yet setting the query hash from their side? If possible, deploying servers // first would work. - AsyncQueryResponse asyncQueryResponse = _asyncQueryResponseMap.get(requestId / Constants.TABLE_TYPE_OFFSET); + AsyncQueryResponse asyncQueryResponse = _asyncQueryResponseMap.get(requestId / BrokerRequestIdConstants.TABLE_TYPE_OFFSET); // Query future might be null if the query is already done (maybe due to failure) if (asyncQueryResponse != null) { diff --git a/pinot-core/src/test/java/org/apache/pinot/core/transport/QueryRoutingTest.java b/pinot-core/src/test/java/org/apache/pinot/core/transport/QueryRoutingTest.java index 317c18a5c4d..52f8f5a9aa5 100644 --- a/pinot-core/src/test/java/org/apache/pinot/core/transport/QueryRoutingTest.java +++ b/pinot-core/src/test/java/org/apache/pinot/core/transport/QueryRoutingTest.java @@ -25,7 +25,7 @@ import java.util.List; import java.util.Map; import org.apache.commons.lang3.tuple.Pair; -import org.apache.pinot.common.Constants; +import org.apache.pinot.common.BrokerRequestIdConstants; import org.apache.pinot.common.datatable.DataTable; import org.apache.pinot.common.datatable.DataTable.MetadataKey; import org.apache.pinot.common.exception.QueryException; @@ -143,7 +143,6 @@ private QueryScheduler mockQueryScheduler(int responseDelayMs, DataTable offline // TODO(egalpin): verify if this arg is tablename String tableName = String.valueOf(((ServerQueryRequest) invocation.getArguments()[0]).getTableName()); TableType tableType = TableNameBuilder.getTableTypeFromTableName(tableName); - AsyncQueryResponse.inferTableType(realtimeDataTable); if (tableName.equals(realtimeDataTable.getMetadata().get(MetadataKey.TABLE.getName())) || tableType.equals( AsyncQueryResponse.inferTableType(realtimeDataTable))) { return Futures.immediateFuture(realtimeDataTable.toBytes()); @@ -162,14 +161,14 @@ public void testValidResponse() long requestId = 1230; DataTable offlineDataTable = DataTableBuilderFactory.getEmptyDataTable(); offlineDataTable.getMetadata() - .put(MetadataKey.REQUEST_ID.getName(), Long.toString(requestId + Constants.OFFLINE_TABLE_DIGIT)); + .put(MetadataKey.REQUEST_ID.getName(), Long.toString(requestId + BrokerRequestIdConstants.OFFLINE_TABLE_DIGIT)); offlineDataTable.getMetadata() .put(MetadataKey.TABLE.getName(), OFFLINE_BROKER_REQUEST.getQuerySource().getTableName()); byte[] offlineResponseBytes = offlineDataTable.toBytes(); DataTable realtimeDataTable = DataTableBuilderFactory.getEmptyDataTable(); realtimeDataTable.getMetadata() - .put(MetadataKey.REQUEST_ID.getName(), Long.toString(requestId + Constants.REALTIME_TABLE_DIGIT)); + .put(MetadataKey.REQUEST_ID.getName(), Long.toString(requestId + BrokerRequestIdConstants.REALTIME_TABLE_DIGIT)); realtimeDataTable.getMetadata() .put(MetadataKey.TABLE.getName(), REALTIME_BROKER_REQUEST.getQuerySource().getTableName()); byte[] realtimeResponseBytes = realtimeDataTable.toBytes(); From 996a6f9e30f77551133b50eda9933a9ba265ed97 Mon Sep 17 00:00:00 2001 From: egalpin Date: Thu, 10 Oct 2024 16:13:10 -0700 Subject: [PATCH 37/49] Moves inferTableType to DataTableUtils --- .../pinot/common/datatable/DataTableUtils.java | 13 +++++++++++++ .../query/reduce/ExecutionStatsAggregator.java | 4 ++-- .../pinot/core/transport/AsyncQueryResponse.java | 15 ++------------- .../pinot/core/transport/QueryRoutingTest.java | 7 ++++--- 4 files changed, 21 insertions(+), 18 deletions(-) diff --git a/pinot-common/src/main/java/org/apache/pinot/common/datatable/DataTableUtils.java b/pinot-common/src/main/java/org/apache/pinot/common/datatable/DataTableUtils.java index dc3b40bfd8a..74ac37f8d0e 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/datatable/DataTableUtils.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/datatable/DataTableUtils.java @@ -21,7 +21,9 @@ import java.io.IOException; import java.nio.ByteBuffer; import org.apache.commons.lang3.StringUtils; +import org.apache.pinot.common.BrokerRequestIdConstants; import org.apache.pinot.common.utils.DataSchema; +import org.apache.pinot.spi.config.table.TableType; import static java.nio.charset.StandardCharsets.UTF_8; @@ -78,4 +80,15 @@ public static String decodeString(ByteBuffer buffer) return new String(bytes, UTF_8); } } + + public static TableType inferTableType(DataTable dataTable) { + TableType tableType; + long requestId = Long.parseLong(dataTable.getMetadata().get(DataTable.MetadataKey.REQUEST_ID.getName())); + if ((requestId & 0x1) == BrokerRequestIdConstants.REALTIME_TABLE_DIGIT) { + tableType = TableType.REALTIME; + } else { + tableType = TableType.OFFLINE; + } + return tableType; + } } diff --git a/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/ExecutionStatsAggregator.java b/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/ExecutionStatsAggregator.java index f946d60afce..757c00981e7 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/ExecutionStatsAggregator.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/query/reduce/ExecutionStatsAggregator.java @@ -25,12 +25,12 @@ import java.util.concurrent.TimeUnit; import java.util.function.LongConsumer; import org.apache.pinot.common.datatable.DataTable; +import org.apache.pinot.common.datatable.DataTableUtils; import org.apache.pinot.common.metrics.BrokerMeter; import org.apache.pinot.common.metrics.BrokerMetrics; import org.apache.pinot.common.metrics.BrokerTimer; import org.apache.pinot.common.response.broker.BrokerResponseNative; import org.apache.pinot.common.response.broker.QueryProcessingException; -import org.apache.pinot.core.transport.AsyncQueryResponse; import org.apache.pinot.core.transport.ServerRoutingInstance; import org.apache.pinot.spi.config.table.TableType; import org.apache.pinot.spi.utils.builder.TableNameBuilder; @@ -79,7 +79,7 @@ public void aggregate(ServerRoutingInstance serverRoutingInstance, DataTable dat if (metadata.get(DataTable.MetadataKey.TABLE.getName()) != null) { tableType = TableNameBuilder.getTableTypeFromTableName(metadata.get(DataTable.MetadataKey.TABLE.getName())); } else if (metadata.get(DataTable.MetadataKey.REQUEST_ID.getName()) != null) { - tableType = AsyncQueryResponse.inferTableType(dataTable); + tableType = DataTableUtils.inferTableType(dataTable); } // Reduce on trace info. diff --git a/pinot-core/src/main/java/org/apache/pinot/core/transport/AsyncQueryResponse.java b/pinot-core/src/main/java/org/apache/pinot/core/transport/AsyncQueryResponse.java index 0b2580c62d8..e01fa84f1fb 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/transport/AsyncQueryResponse.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/transport/AsyncQueryResponse.java @@ -29,8 +29,8 @@ import java.util.concurrent.atomic.AtomicReference; import javax.annotation.Nullable; import javax.annotation.concurrent.ThreadSafe; -import org.apache.pinot.common.BrokerRequestIdConstants; import org.apache.pinot.common.datatable.DataTable; +import org.apache.pinot.common.datatable.DataTableUtils; import org.apache.pinot.common.exception.QueryException; import org.apache.pinot.common.request.InstanceRequest; import org.apache.pinot.core.transport.server.routing.stats.ServerRoutingStatsManager; @@ -225,20 +225,9 @@ void markRequestSent(ServerRoutingInstance serverRoutingInstance, InstanceReques .markRequestSent(requestSentLatencyMs); } - public static TableType inferTableType(DataTable dataTable) { - TableType tableType; - long requestId = Long.parseLong(dataTable.getMetadata().get(DataTable.MetadataKey.REQUEST_ID.getName())); - if ((requestId & 0x1) == BrokerRequestIdConstants.REALTIME_TABLE_DIGIT) { - tableType = TableType.REALTIME; - } else { - tableType = TableType.OFFLINE; - } - return tableType; - } - @Nullable private String inferTableName(ServerRoutingInstance serverRoutingInstance, DataTable dataTable) { - TableType tableType = inferTableType(dataTable); + TableType tableType = DataTableUtils.inferTableType(dataTable); for (Map.Entry serverResponseEntry : _responses.get(serverRoutingInstance).entrySet()) { String tableName = serverResponseEntry.getKey(); diff --git a/pinot-core/src/test/java/org/apache/pinot/core/transport/QueryRoutingTest.java b/pinot-core/src/test/java/org/apache/pinot/core/transport/QueryRoutingTest.java index 52f8f5a9aa5..03b0fee693d 100644 --- a/pinot-core/src/test/java/org/apache/pinot/core/transport/QueryRoutingTest.java +++ b/pinot-core/src/test/java/org/apache/pinot/core/transport/QueryRoutingTest.java @@ -28,6 +28,7 @@ import org.apache.pinot.common.BrokerRequestIdConstants; import org.apache.pinot.common.datatable.DataTable; import org.apache.pinot.common.datatable.DataTable.MetadataKey; +import org.apache.pinot.common.datatable.DataTableUtils; import org.apache.pinot.common.exception.QueryException; import org.apache.pinot.common.metrics.BrokerMetrics; import org.apache.pinot.common.metrics.ServerMetrics; @@ -144,10 +145,10 @@ private QueryScheduler mockQueryScheduler(int responseDelayMs, DataTable offline String tableName = String.valueOf(((ServerQueryRequest) invocation.getArguments()[0]).getTableName()); TableType tableType = TableNameBuilder.getTableTypeFromTableName(tableName); if (tableName.equals(realtimeDataTable.getMetadata().get(MetadataKey.TABLE.getName())) || tableType.equals( - AsyncQueryResponse.inferTableType(realtimeDataTable))) { + DataTableUtils.inferTableType(realtimeDataTable))) { return Futures.immediateFuture(realtimeDataTable.toBytes()); } else if (tableName.equals(offlineDataTable.getMetadata().get(MetadataKey.TABLE.getName())) || tableType.equals( - AsyncQueryResponse.inferTableType(offlineDataTable))) { + DataTableUtils.inferTableType(offlineDataTable))) { return Futures.immediateFuture(offlineDataTable.toBytes()); } return Futures.immediateFuture(new byte[0]); @@ -202,7 +203,7 @@ public void testValidResponse() assertEquals(response.get(SERVER_ROUTING_INSTANCE).size(), 1); serverResponse = response.get(SERVER_ROUTING_INSTANCE).get(0); assertNotNull(serverResponse.getDataTable()); - assertEquals(TableType.REALTIME, AsyncQueryResponse.inferTableType(serverResponse.getDataTable())); + assertEquals(TableType.REALTIME, DataTableUtils.inferTableType(serverResponse.getDataTable())); assertEquals(serverResponse.getResponseSize(), realtimeResponseBytes.length); _requestCount += 2; waitForStatsUpdate(_requestCount); From eb10a8b5554e0f4494607851dc05e131efd6d90f Mon Sep 17 00:00:00 2001 From: egalpin Date: Thu, 10 Oct 2024 16:19:06 -0700 Subject: [PATCH 38/49] Clean up references to tableName --- .../pinot/core/query/request/ServerQueryRequest.java | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/pinot-core/src/main/java/org/apache/pinot/core/query/request/ServerQueryRequest.java b/pinot-core/src/main/java/org/apache/pinot/core/query/request/ServerQueryRequest.java index 465397838a9..98fcdb2d584 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/query/request/ServerQueryRequest.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/query/request/ServerQueryRequest.java @@ -75,8 +75,8 @@ public ServerQueryRequest(InstanceRequest instanceRequest, ServerMetrics serverM // Method to set table name needs to match whats in AsyncQueryResponse _tableName = _queryContext.getTableName(); _queryId = QueryIdUtils.getQueryId(_brokerId, _requestId, - TableNameBuilder.getTableTypeFromTableName(_queryContext.getTableName())); - _timerContext = new TimerContext(_queryContext.getTableName(), serverMetrics, queryArrivalTimeMs); + TableNameBuilder.getTableTypeFromTableName(_tableName)); + _timerContext = new TimerContext(_tableName, serverMetrics, queryArrivalTimeMs); } public ServerQueryRequest(Server.ServerRequest serverRequest, ServerMetrics serverMetrics) @@ -107,8 +107,8 @@ public ServerQueryRequest(Server.ServerRequest serverRequest, ServerMetrics serv _queryContext = getQueryContext(brokerRequest.getPinotQuery()); _tableName = _queryContext.getTableName(); _queryId = QueryIdUtils.getQueryId(_brokerId, _requestId, - TableNameBuilder.getTableTypeFromTableName(_queryContext.getTableName())); - _timerContext = new TimerContext(_queryContext.getTableName(), serverMetrics, queryArrivalTimeMs); + TableNameBuilder.getTableTypeFromTableName(_tableName)); + _timerContext = new TimerContext(_tableName, serverMetrics, queryArrivalTimeMs); } /** @@ -131,7 +131,7 @@ public ServerQueryRequest(QueryContext queryContext, List segmentsToQuer _segmentsToQuery = segmentsToQuery; _optionalSegments = null; - _timerContext = new TimerContext(_queryContext.getTableName(), serverMetrics, queryArrivalTimeMs); + _timerContext = new TimerContext(_tableName, serverMetrics, queryArrivalTimeMs); } private static QueryContext getQueryContext(PinotQuery pinotQuery) { From a9c8d2e00a0ce7756658234f88bf197c4da2e975 Mon Sep 17 00:00:00 2001 From: egalpin Date: Thu, 10 Oct 2024 16:22:36 -0700 Subject: [PATCH 39/49] Updates PrioritySchedulerTest#testSubmitBeforeRunning to expect that server always sends tableName in meta --- .../pinot/core/query/scheduler/PrioritySchedulerTest.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/pinot-core/src/test/java/org/apache/pinot/core/query/scheduler/PrioritySchedulerTest.java b/pinot-core/src/test/java/org/apache/pinot/core/query/scheduler/PrioritySchedulerTest.java index 5f92dd5a25a..ebc6c86e1cc 100644 --- a/pinot-core/src/test/java/org/apache/pinot/core/query/scheduler/PrioritySchedulerTest.java +++ b/pinot-core/src/test/java/org/apache/pinot/core/query/scheduler/PrioritySchedulerTest.java @@ -225,7 +225,8 @@ public void testSubmitBeforeRunning() // start is not called DataTable response = DataTableFactory.getDataTable(result.get()); assertTrue(response.getExceptions().containsKey(QueryException.SERVER_SCHEDULER_DOWN_ERROR.getErrorCode())); - assertFalse(response.getMetadata().containsKey(MetadataKey.TABLE.getName())); + // Server should always try to return the table name to which the query pertained + assertTrue(response.getMetadata().containsKey(MetadataKey.TABLE.getName())); scheduler.stop(); } From 734189bc7da58705f6090102abbec76758ab84ee Mon Sep 17 00:00:00 2001 From: egalpin Date: Thu, 10 Oct 2024 16:23:46 -0700 Subject: [PATCH 40/49] Linting fixes --- .../broker/requesthandler/BrokerRequestIdGenerator.java | 5 +++-- .../requesthandler/BrokerRequestIdGeneratorTest.java | 9 +++++---- .../org/apache/pinot/core/transport/QueryRouter.java | 8 ++++---- .../apache/pinot/core/transport/QueryRoutingTest.java | 4 ++-- 4 files changed, 14 insertions(+), 12 deletions(-) diff --git a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BrokerRequestIdGenerator.java b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BrokerRequestIdGenerator.java index 1a2560ecb98..eedd458e3c2 100644 --- a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BrokerRequestIdGenerator.java +++ b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BrokerRequestIdGenerator.java @@ -45,8 +45,9 @@ public BrokerRequestIdGenerator(String brokerId) { } public long get() { - long normalized = ((_incrementingId.getAndIncrement() & Long.MAX_VALUE) % (OFFSET / BrokerRequestIdConstants.TABLE_TYPE_OFFSET)) - * BrokerRequestIdConstants.TABLE_TYPE_OFFSET; + long normalized = + ((_incrementingId.getAndIncrement() & Long.MAX_VALUE) % (OFFSET / BrokerRequestIdConstants.TABLE_TYPE_OFFSET)) + * BrokerRequestIdConstants.TABLE_TYPE_OFFSET; return _mask + normalized; } } diff --git a/pinot-broker/src/test/java/org/apache/pinot/broker/requesthandler/BrokerRequestIdGeneratorTest.java b/pinot-broker/src/test/java/org/apache/pinot/broker/requesthandler/BrokerRequestIdGeneratorTest.java index ee6efcaa2e9..3340ed4c0cf 100644 --- a/pinot-broker/src/test/java/org/apache/pinot/broker/requesthandler/BrokerRequestIdGeneratorTest.java +++ b/pinot-broker/src/test/java/org/apache/pinot/broker/requesthandler/BrokerRequestIdGeneratorTest.java @@ -18,6 +18,7 @@ */ package org.apache.pinot.broker.requesthandler; +import org.apache.pinot.common.BrokerRequestIdConstants; import org.testng.annotations.Test; import static org.testng.Assert.*; @@ -29,11 +30,11 @@ public class BrokerRequestIdGeneratorTest { public void testGet() { BrokerRequestIdGenerator gen = new BrokerRequestIdGenerator("foo"); long id = gen.get(); - assertEquals(id % 10, 0); - assertEquals(id / 10 % 10, 0); + assertEquals(id % BrokerRequestIdConstants.TABLE_TYPE_OFFSET, 0); + assertEquals(id / BrokerRequestIdConstants.TABLE_TYPE_OFFSET % BrokerRequestIdConstants.TABLE_TYPE_OFFSET, 0); id = gen.get(); - assertEquals(id % 10, 0); - assertEquals(id / 10 % 10, 1); + assertEquals(id % BrokerRequestIdConstants.TABLE_TYPE_OFFSET, 0); + assertEquals(id / BrokerRequestIdConstants.TABLE_TYPE_OFFSET % BrokerRequestIdConstants.TABLE_TYPE_OFFSET, 1); } } diff --git a/pinot-core/src/main/java/org/apache/pinot/core/transport/QueryRouter.java b/pinot-core/src/main/java/org/apache/pinot/core/transport/QueryRouter.java index 1d7174054a1..261855bb5d0 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/transport/QueryRouter.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/transport/QueryRouter.java @@ -106,9 +106,8 @@ public AsyncQueryResponse submitQuery(long requestId, String rawTableName, ServerRoutingInstance serverRoutingInstance = entry.getKey().toServerRoutingInstance(preferTls); // TODO(egalpin): inject value into request ID to indicate REALTIME Vs OFFLINE table type? - long tableTypeDigit = - serverQueryRoutingContext.getTableType().equals(TableType.OFFLINE) ? BrokerRequestIdConstants.OFFLINE_TABLE_DIGIT - : BrokerRequestIdConstants.REALTIME_TABLE_DIGIT; + long tableTypeDigit = serverQueryRoutingContext.getTableType().equals(TableType.OFFLINE) + ? BrokerRequestIdConstants.OFFLINE_TABLE_DIGIT : BrokerRequestIdConstants.REALTIME_TABLE_DIGIT; InstanceRequest instanceRequest = getInstanceRequest(requestId + tableTypeDigit, serverQueryRoutingContext.getBrokerRequest(), @@ -199,7 +198,8 @@ void receiveDataTable(ServerRoutingInstance serverRoutingInstance, DataTable dat // TODO(egalpin): How can we handle rolling out brokers expecting query_hash to be present, but while old server // versions are still running and not yet setting the query hash from their side? If possible, deploying servers // first would work. - AsyncQueryResponse asyncQueryResponse = _asyncQueryResponseMap.get(requestId / BrokerRequestIdConstants.TABLE_TYPE_OFFSET); + AsyncQueryResponse asyncQueryResponse = + _asyncQueryResponseMap.get(requestId / BrokerRequestIdConstants.TABLE_TYPE_OFFSET); // Query future might be null if the query is already done (maybe due to failure) if (asyncQueryResponse != null) { diff --git a/pinot-core/src/test/java/org/apache/pinot/core/transport/QueryRoutingTest.java b/pinot-core/src/test/java/org/apache/pinot/core/transport/QueryRoutingTest.java index 03b0fee693d..5d291e73d9f 100644 --- a/pinot-core/src/test/java/org/apache/pinot/core/transport/QueryRoutingTest.java +++ b/pinot-core/src/test/java/org/apache/pinot/core/transport/QueryRoutingTest.java @@ -168,8 +168,8 @@ public void testValidResponse() byte[] offlineResponseBytes = offlineDataTable.toBytes(); DataTable realtimeDataTable = DataTableBuilderFactory.getEmptyDataTable(); - realtimeDataTable.getMetadata() - .put(MetadataKey.REQUEST_ID.getName(), Long.toString(requestId + BrokerRequestIdConstants.REALTIME_TABLE_DIGIT)); + realtimeDataTable.getMetadata().put(MetadataKey.REQUEST_ID.getName(), + Long.toString(requestId + BrokerRequestIdConstants.REALTIME_TABLE_DIGIT)); realtimeDataTable.getMetadata() .put(MetadataKey.TABLE.getName(), REALTIME_BROKER_REQUEST.getQuerySource().getTableName()); byte[] realtimeResponseBytes = realtimeDataTable.toBytes(); From 4d621fa2879185cc166ed210df52c121bea4d2f9 Mon Sep 17 00:00:00 2001 From: egalpin Date: Fri, 11 Oct 2024 10:05:23 -0700 Subject: [PATCH 41/49] Empty commit retrigger tests From 41aa2e1260c2112b515059ba073b12ac788a4ae2 Mon Sep 17 00:00:00 2001 From: egalpin Date: Thu, 17 Oct 2024 11:30:27 -0700 Subject: [PATCH 42/49] Makes requestId manipulations idempotent --- .../BrokerRequestIdGenerator.java | 8 +++--- .../BrokerRequestIdGeneratorTest.java | 10 +++---- .../common/datatable/DataTableUtils.java | 4 +-- .../request/BrokerRequestIdUtils.java} | 21 ++++++++++---- .../request/BrokerRequestIdUtilsTest.java | 22 +++++++++++++++ .../core/transport/AsyncQueryResponse.java | 16 +++++------ .../pinot/core/transport/QueryRouter.java | 28 ++++++++----------- .../core/transport/QueryRoutingTest.java | 8 +++--- 8 files changed, 71 insertions(+), 46 deletions(-) rename pinot-common/src/main/java/org/apache/pinot/common/{BrokerRequestIdConstants.java => utils/request/BrokerRequestIdUtils.java} (64%) create mode 100644 pinot-common/src/test/java/org/apache/pinot/common/utils/request/BrokerRequestIdUtilsTest.java diff --git a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BrokerRequestIdGenerator.java b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BrokerRequestIdGenerator.java index eedd458e3c2..1faffcaa905 100644 --- a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BrokerRequestIdGenerator.java +++ b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/BrokerRequestIdGenerator.java @@ -19,7 +19,7 @@ package org.apache.pinot.broker.requesthandler; import java.util.concurrent.atomic.AtomicLong; -import org.apache.pinot.common.BrokerRequestIdConstants; +import org.apache.pinot.common.utils.request.BrokerRequestIdUtils; /** @@ -46,8 +46,8 @@ public BrokerRequestIdGenerator(String brokerId) { public long get() { long normalized = - ((_incrementingId.getAndIncrement() & Long.MAX_VALUE) % (OFFSET / BrokerRequestIdConstants.TABLE_TYPE_OFFSET)) - * BrokerRequestIdConstants.TABLE_TYPE_OFFSET; - return _mask + normalized; + ((_incrementingId.getAndIncrement() & Long.MAX_VALUE) % (OFFSET / BrokerRequestIdUtils.TABLE_TYPE_OFFSET)) + * BrokerRequestIdUtils.TABLE_TYPE_OFFSET; + return BrokerRequestIdUtils.getCanonicalRequestId(_mask + normalized); } } diff --git a/pinot-broker/src/test/java/org/apache/pinot/broker/requesthandler/BrokerRequestIdGeneratorTest.java b/pinot-broker/src/test/java/org/apache/pinot/broker/requesthandler/BrokerRequestIdGeneratorTest.java index 3340ed4c0cf..c4bea7cc2aa 100644 --- a/pinot-broker/src/test/java/org/apache/pinot/broker/requesthandler/BrokerRequestIdGeneratorTest.java +++ b/pinot-broker/src/test/java/org/apache/pinot/broker/requesthandler/BrokerRequestIdGeneratorTest.java @@ -18,7 +18,7 @@ */ package org.apache.pinot.broker.requesthandler; -import org.apache.pinot.common.BrokerRequestIdConstants; +import org.apache.pinot.common.utils.request.BrokerRequestIdUtils; import org.testng.annotations.Test; import static org.testng.Assert.*; @@ -30,11 +30,11 @@ public class BrokerRequestIdGeneratorTest { public void testGet() { BrokerRequestIdGenerator gen = new BrokerRequestIdGenerator("foo"); long id = gen.get(); - assertEquals(id % BrokerRequestIdConstants.TABLE_TYPE_OFFSET, 0); - assertEquals(id / BrokerRequestIdConstants.TABLE_TYPE_OFFSET % BrokerRequestIdConstants.TABLE_TYPE_OFFSET, 0); + assertEquals(id % BrokerRequestIdUtils.TABLE_TYPE_OFFSET, 0); + assertEquals(id / BrokerRequestIdUtils.TABLE_TYPE_OFFSET % BrokerRequestIdUtils.TABLE_TYPE_OFFSET, 0); id = gen.get(); - assertEquals(id % BrokerRequestIdConstants.TABLE_TYPE_OFFSET, 0); - assertEquals(id / BrokerRequestIdConstants.TABLE_TYPE_OFFSET % BrokerRequestIdConstants.TABLE_TYPE_OFFSET, 1); + assertEquals(id % BrokerRequestIdUtils.TABLE_TYPE_OFFSET, 0); + assertEquals(id / BrokerRequestIdUtils.TABLE_TYPE_OFFSET % BrokerRequestIdUtils.TABLE_TYPE_OFFSET, 1); } } diff --git a/pinot-common/src/main/java/org/apache/pinot/common/datatable/DataTableUtils.java b/pinot-common/src/main/java/org/apache/pinot/common/datatable/DataTableUtils.java index 74ac37f8d0e..a91958fd18f 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/datatable/DataTableUtils.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/datatable/DataTableUtils.java @@ -21,7 +21,7 @@ import java.io.IOException; import java.nio.ByteBuffer; import org.apache.commons.lang3.StringUtils; -import org.apache.pinot.common.BrokerRequestIdConstants; +import org.apache.pinot.common.utils.request.BrokerRequestIdUtils; import org.apache.pinot.common.utils.DataSchema; import org.apache.pinot.spi.config.table.TableType; @@ -84,7 +84,7 @@ public static String decodeString(ByteBuffer buffer) public static TableType inferTableType(DataTable dataTable) { TableType tableType; long requestId = Long.parseLong(dataTable.getMetadata().get(DataTable.MetadataKey.REQUEST_ID.getName())); - if ((requestId & 0x1) == BrokerRequestIdConstants.REALTIME_TABLE_DIGIT) { + if (requestId == BrokerRequestIdUtils.getRealtimeRequestId(requestId)) { tableType = TableType.REALTIME; } else { tableType = TableType.OFFLINE; diff --git a/pinot-common/src/main/java/org/apache/pinot/common/BrokerRequestIdConstants.java b/pinot-common/src/main/java/org/apache/pinot/common/utils/request/BrokerRequestIdUtils.java similarity index 64% rename from pinot-common/src/main/java/org/apache/pinot/common/BrokerRequestIdConstants.java rename to pinot-common/src/main/java/org/apache/pinot/common/utils/request/BrokerRequestIdUtils.java index c14940f69e4..a01a2a8798d 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/BrokerRequestIdConstants.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/utils/request/BrokerRequestIdUtils.java @@ -16,12 +16,23 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.pinot.common; +package org.apache.pinot.common.utils.request; -public class BrokerRequestIdConstants { +public class BrokerRequestIdUtils { public static final int TABLE_TYPE_OFFSET = 10; - public static final int OFFLINE_TABLE_DIGIT = 0; - public static final int REALTIME_TABLE_DIGIT = 1; - private BrokerRequestIdConstants() { + + private BrokerRequestIdUtils() { + } + + public static long getOfflineRequestId(long requestId) { + return getCanonicalRequestId(requestId); + } + + public static long getRealtimeRequestId(long requestId) { + return getCanonicalRequestId(requestId) | 0x1; + } + + public static long getCanonicalRequestId(long requestId) { + return requestId / TABLE_TYPE_OFFSET * TABLE_TYPE_OFFSET; } } diff --git a/pinot-common/src/test/java/org/apache/pinot/common/utils/request/BrokerRequestIdUtilsTest.java b/pinot-common/src/test/java/org/apache/pinot/common/utils/request/BrokerRequestIdUtilsTest.java new file mode 100644 index 00000000000..c80f167443c --- /dev/null +++ b/pinot-common/src/test/java/org/apache/pinot/common/utils/request/BrokerRequestIdUtilsTest.java @@ -0,0 +1,22 @@ +package org.apache.pinot.common.utils.request; + +import org.testng.Assert; +import org.testng.annotations.Test; + + +public class BrokerRequestIdUtilsTest { + @Test + public void testGetOfflineRequestId() { + Assert.assertEquals(BrokerRequestIdUtils.getOfflineRequestId(123), 120); + } + + @Test + public void testGetRealtimeRequestId() { + Assert.assertEquals(BrokerRequestIdUtils.getRealtimeRequestId(123), 121); + } + + @Test + public void testGetCanonicalRequestId() { + Assert.assertEquals(BrokerRequestIdUtils.getCanonicalRequestId(123), 120); + } +} diff --git a/pinot-core/src/main/java/org/apache/pinot/core/transport/AsyncQueryResponse.java b/pinot-core/src/main/java/org/apache/pinot/core/transport/AsyncQueryResponse.java index e01fa84f1fb..6a78162347c 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/transport/AsyncQueryResponse.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/transport/AsyncQueryResponse.java @@ -47,7 +47,7 @@ public class AsyncQueryResponse implements QueryResponse { // TODO(egalpin): Remove the concept of truncated request ID in next major release after all servers are expected // to send back tableName in DataTable metadata - private final long _truncatedRequestId; + private final long _canonicalRequestId; private final AtomicReference _status = new AtomicReference<>(Status.IN_PROGRESS); private final AtomicInteger _numServersResponded = new AtomicInteger(); private final ConcurrentHashMap> _responses; @@ -59,11 +59,11 @@ public class AsyncQueryResponse implements QueryResponse { private volatile ServerRoutingInstance _failedServer; private volatile Exception _exception; - public AsyncQueryResponse(QueryRouter queryRouter, long truncatedRequestId, + public AsyncQueryResponse(QueryRouter queryRouter, long canonicalRequestId, Map> requestMap, long startTimeMs, long timeoutMs, ServerRoutingStatsManager serverRoutingStatsManager) { _queryRouter = queryRouter; - _truncatedRequestId = truncatedRequestId; + _canonicalRequestId = canonicalRequestId; _responses = new ConcurrentHashMap<>(); _serverRoutingStatsManager = serverRoutingStatsManager; int numQueriesIssued = 0; @@ -72,7 +72,7 @@ public AsyncQueryResponse(QueryRouter queryRouter, long truncatedRequestId, // Record stats related to query submission just before sending the request. Otherwise, if the response is // received immediately, there's a possibility of updating query response stats before updating query // submission stats. - _serverRoutingStatsManager.recordStatsForQuerySubmission(_truncatedRequestId, + _serverRoutingStatsManager.recordStatsForQuerySubmission(_canonicalRequestId, serverRequests.getKey().getInstanceId()); _responses.computeIfAbsent(serverRequests.getKey(), k -> new ConcurrentHashMap<>()) @@ -136,11 +136,11 @@ public Map> getFinalResponses() } else { latency = response.getResponseDelayMs(); } - _serverRoutingStatsManager.recordStatsUponResponseArrival(_truncatedRequestId, + _serverRoutingStatsManager.recordStatsUponResponseArrival(_canonicalRequestId, serverResponses.getKey().getInstanceId(), latency); } } - _queryRouter.markQueryDone(_truncatedRequestId); + _queryRouter.markQueryDone(_canonicalRequestId); } } @@ -206,7 +206,7 @@ public Exception getException() { @Override public long getRequestId() { - return _truncatedRequestId; + return _canonicalRequestId; } @Override @@ -272,8 +272,6 @@ void markQueryFailed(ServerRoutingInstance serverRoutingInstance, Exception exce * server hasn't responded yet. */ void markServerDown(ServerRoutingInstance serverRoutingInstance, Exception exception) { - // TODO(egalpin): how to mark servers down under the assumption that multiple queries - // to the same server are valid? Map serverResponses = _responses.get(serverRoutingInstance); for (ServerResponse serverResponse : serverResponses.values()) { if (serverResponse != null && serverResponse.getDataTable() == null) { diff --git a/pinot-core/src/main/java/org/apache/pinot/core/transport/QueryRouter.java b/pinot-core/src/main/java/org/apache/pinot/core/transport/QueryRouter.java index 261855bb5d0..d26440169b0 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/transport/QueryRouter.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/transport/QueryRouter.java @@ -28,7 +28,7 @@ import javax.annotation.concurrent.ThreadSafe; import org.apache.commons.collections4.CollectionUtils; import org.apache.commons.lang3.tuple.Pair; -import org.apache.pinot.common.BrokerRequestIdConstants; +import org.apache.pinot.common.utils.request.BrokerRequestIdUtils; import org.apache.pinot.common.config.NettyConfig; import org.apache.pinot.common.config.TlsConfig; import org.apache.pinot.common.datatable.DataTable; @@ -105,12 +105,12 @@ public AsyncQueryResponse submitQuery(long requestId, String rawTableName, for (ServerQueryRoutingContext serverQueryRoutingContext : entry.getValue()) { ServerRoutingInstance serverRoutingInstance = entry.getKey().toServerRoutingInstance(preferTls); - // TODO(egalpin): inject value into request ID to indicate REALTIME Vs OFFLINE table type? - long tableTypeDigit = serverQueryRoutingContext.getTableType().equals(TableType.OFFLINE) - ? BrokerRequestIdConstants.OFFLINE_TABLE_DIGIT : BrokerRequestIdConstants.REALTIME_TABLE_DIGIT; + long requestIdWithType = serverQueryRoutingContext.getTableType().equals(TableType.OFFLINE) + ? BrokerRequestIdUtils.getOfflineRequestId(requestId) + : BrokerRequestIdUtils.getRealtimeRequestId(requestId); InstanceRequest instanceRequest = - getInstanceRequest(requestId + tableTypeDigit, serverQueryRoutingContext.getBrokerRequest(), + getInstanceRequest(requestIdWithType, serverQueryRoutingContext.getBrokerRequest(), serverQueryRoutingContext.getSegmentsToQuery()); if (!skipUnavailableServers && QueryOptionsUtils.isSkipUnavailableServers( serverQueryRoutingContext.getBrokerRequest().getPinotQuery().getQueryOptions())) { @@ -123,16 +123,13 @@ public AsyncQueryResponse submitQuery(long requestId, String rawTableName, } } - // Create the asynchronous query response with the request map - long truncatedRequestId = requestId / BrokerRequestIdConstants.TABLE_TYPE_OFFSET; - // Map the request using all but the last digit. This way, servers will return request IDs where the last digit // (injected above) will indicate table type, but both will be able to be divided by the TABLE_TYPE_OFFSET in // order to properly map back to this originating request AsyncQueryResponse asyncQueryResponse = - new AsyncQueryResponse(this, truncatedRequestId, requestMap, System.currentTimeMillis(), timeoutMs, + new AsyncQueryResponse(this, requestId, requestMap, System.currentTimeMillis(), timeoutMs, _serverRoutingStatsManager); - _asyncQueryResponseMap.put(truncatedRequestId, asyncQueryResponse); + _asyncQueryResponseMap.put(requestId, asyncQueryResponse); for (Map.Entry> serverRequests : requestMap.entrySet()) { for (InstanceRequest request : serverRequests.getValue()) { @@ -145,14 +142,14 @@ public AsyncQueryResponse submitQuery(long requestId, String rawTableName, if (ServerChannels.CHANNEL_LOCK_TIMEOUT_MSG.equals(e.getMessage())) { _brokerMetrics.addMeteredTableValue(rawTableName, BrokerMeter.REQUEST_CHANNEL_LOCK_TIMEOUT_EXCEPTIONS, 1); } - markQueryFailed(truncatedRequestId, serverRoutingInstance, asyncQueryResponse, e); + markQueryFailed(requestId, serverRoutingInstance, asyncQueryResponse, e); break; } catch (Exception e) { _brokerMetrics.addMeteredTableValue(rawTableName, BrokerMeter.REQUEST_SEND_EXCEPTIONS, 1); if (skipUnavailableServers) { asyncQueryResponse.skipServerResponse(); } else { - markQueryFailed(truncatedRequestId, serverRoutingInstance, asyncQueryResponse, e); + markQueryFailed(requestId, serverRoutingInstance, asyncQueryResponse, e); break; } } @@ -193,13 +190,10 @@ public void shutDown() { void receiveDataTable(ServerRoutingInstance serverRoutingInstance, DataTable dataTable, int responseSize, int deserializationTimeMs) { long requestId = Long.parseLong(dataTable.getMetadata().get(MetadataKey.REQUEST_ID.getName())); - // TODO(egalpin): parse out table type based on last digit of request ID? - // TODO(egalpin): How can we handle rolling out brokers expecting query_hash to be present, but while old server - // versions are still running and not yet setting the query hash from their side? If possible, deploying servers - // first would work. + // TODO(egalpin): remove use of canonical request ID in later major release AsyncQueryResponse asyncQueryResponse = - _asyncQueryResponseMap.get(requestId / BrokerRequestIdConstants.TABLE_TYPE_OFFSET); + _asyncQueryResponseMap.get(BrokerRequestIdUtils.getCanonicalRequestId(requestId)); // Query future might be null if the query is already done (maybe due to failure) if (asyncQueryResponse != null) { diff --git a/pinot-core/src/test/java/org/apache/pinot/core/transport/QueryRoutingTest.java b/pinot-core/src/test/java/org/apache/pinot/core/transport/QueryRoutingTest.java index 5d291e73d9f..22dd2807dfa 100644 --- a/pinot-core/src/test/java/org/apache/pinot/core/transport/QueryRoutingTest.java +++ b/pinot-core/src/test/java/org/apache/pinot/core/transport/QueryRoutingTest.java @@ -25,7 +25,7 @@ import java.util.List; import java.util.Map; import org.apache.commons.lang3.tuple.Pair; -import org.apache.pinot.common.BrokerRequestIdConstants; +import org.apache.pinot.common.utils.request.BrokerRequestIdUtils; import org.apache.pinot.common.datatable.DataTable; import org.apache.pinot.common.datatable.DataTable.MetadataKey; import org.apache.pinot.common.datatable.DataTableUtils; @@ -161,15 +161,15 @@ public void testValidResponse() throws Exception { long requestId = 1230; DataTable offlineDataTable = DataTableBuilderFactory.getEmptyDataTable(); - offlineDataTable.getMetadata() - .put(MetadataKey.REQUEST_ID.getName(), Long.toString(requestId + BrokerRequestIdConstants.OFFLINE_TABLE_DIGIT)); + offlineDataTable.getMetadata().put(MetadataKey.REQUEST_ID.getName(), + Long.toString(BrokerRequestIdUtils.getOfflineRequestId(requestId))); offlineDataTable.getMetadata() .put(MetadataKey.TABLE.getName(), OFFLINE_BROKER_REQUEST.getQuerySource().getTableName()); byte[] offlineResponseBytes = offlineDataTable.toBytes(); DataTable realtimeDataTable = DataTableBuilderFactory.getEmptyDataTable(); realtimeDataTable.getMetadata().put(MetadataKey.REQUEST_ID.getName(), - Long.toString(requestId + BrokerRequestIdConstants.REALTIME_TABLE_DIGIT)); + Long.toString(BrokerRequestIdUtils.getRealtimeRequestId(requestId))); realtimeDataTable.getMetadata() .put(MetadataKey.TABLE.getName(), REALTIME_BROKER_REQUEST.getQuerySource().getTableName()); byte[] realtimeResponseBytes = realtimeDataTable.toBytes(); From edc5aa4008f81b873f97d0d8960973d66c169d27 Mon Sep 17 00:00:00 2001 From: egalpin Date: Thu, 17 Oct 2024 12:56:55 -0700 Subject: [PATCH 43/49] Adds license to BrokerRequestIdUtilsTest.java --- .../request/BrokerRequestIdUtilsTest.java | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) diff --git a/pinot-common/src/test/java/org/apache/pinot/common/utils/request/BrokerRequestIdUtilsTest.java b/pinot-common/src/test/java/org/apache/pinot/common/utils/request/BrokerRequestIdUtilsTest.java index c80f167443c..1f05e870542 100644 --- a/pinot-common/src/test/java/org/apache/pinot/common/utils/request/BrokerRequestIdUtilsTest.java +++ b/pinot-common/src/test/java/org/apache/pinot/common/utils/request/BrokerRequestIdUtilsTest.java @@ -1,3 +1,21 @@ +/** + * 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.common.utils.request; import org.testng.Assert; From 86c1d4c6ba41491ee4f62af8c920160812cddde8 Mon Sep 17 00:00:00 2001 From: egalpin Date: Thu, 17 Oct 2024 13:11:35 -0700 Subject: [PATCH 44/49] Fixes spotless violations --- .../java/org/apache/pinot/common/datatable/DataTableUtils.java | 2 +- .../main/java/org/apache/pinot/core/transport/QueryRouter.java | 2 +- .../java/org/apache/pinot/core/transport/QueryRoutingTest.java | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/pinot-common/src/main/java/org/apache/pinot/common/datatable/DataTableUtils.java b/pinot-common/src/main/java/org/apache/pinot/common/datatable/DataTableUtils.java index a91958fd18f..37ec424baee 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/datatable/DataTableUtils.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/datatable/DataTableUtils.java @@ -21,8 +21,8 @@ import java.io.IOException; import java.nio.ByteBuffer; import org.apache.commons.lang3.StringUtils; -import org.apache.pinot.common.utils.request.BrokerRequestIdUtils; import org.apache.pinot.common.utils.DataSchema; +import org.apache.pinot.common.utils.request.BrokerRequestIdUtils; import org.apache.pinot.spi.config.table.TableType; import static java.nio.charset.StandardCharsets.UTF_8; diff --git a/pinot-core/src/main/java/org/apache/pinot/core/transport/QueryRouter.java b/pinot-core/src/main/java/org/apache/pinot/core/transport/QueryRouter.java index d26440169b0..145ce57bcac 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/transport/QueryRouter.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/transport/QueryRouter.java @@ -28,7 +28,6 @@ import javax.annotation.concurrent.ThreadSafe; import org.apache.commons.collections4.CollectionUtils; import org.apache.commons.lang3.tuple.Pair; -import org.apache.pinot.common.utils.request.BrokerRequestIdUtils; import org.apache.pinot.common.config.NettyConfig; import org.apache.pinot.common.config.TlsConfig; import org.apache.pinot.common.datatable.DataTable; @@ -38,6 +37,7 @@ import org.apache.pinot.common.request.BrokerRequest; import org.apache.pinot.common.request.InstanceRequest; import org.apache.pinot.common.utils.config.QueryOptionsUtils; +import org.apache.pinot.common.utils.request.BrokerRequestIdUtils; import org.apache.pinot.core.transport.server.routing.stats.ServerRoutingStatsManager; import org.apache.pinot.spi.config.table.TableType; import org.apache.pinot.spi.utils.CommonConstants; diff --git a/pinot-core/src/test/java/org/apache/pinot/core/transport/QueryRoutingTest.java b/pinot-core/src/test/java/org/apache/pinot/core/transport/QueryRoutingTest.java index 22dd2807dfa..b3371d8111f 100644 --- a/pinot-core/src/test/java/org/apache/pinot/core/transport/QueryRoutingTest.java +++ b/pinot-core/src/test/java/org/apache/pinot/core/transport/QueryRoutingTest.java @@ -25,7 +25,6 @@ import java.util.List; import java.util.Map; import org.apache.commons.lang3.tuple.Pair; -import org.apache.pinot.common.utils.request.BrokerRequestIdUtils; import org.apache.pinot.common.datatable.DataTable; import org.apache.pinot.common.datatable.DataTable.MetadataKey; import org.apache.pinot.common.datatable.DataTableUtils; @@ -35,6 +34,7 @@ import org.apache.pinot.common.request.BrokerRequest; import org.apache.pinot.common.response.ProcessingException; import org.apache.pinot.common.utils.DataSchema; +import org.apache.pinot.common.utils.request.BrokerRequestIdUtils; import org.apache.pinot.core.common.datatable.DataTableBuilder; import org.apache.pinot.core.common.datatable.DataTableBuilderFactory; import org.apache.pinot.core.query.request.ServerQueryRequest; From 2ec099bad226c7f164ae83ce55d2073789fc490e Mon Sep 17 00:00:00 2001 From: egalpin Date: Thu, 17 Oct 2024 14:08:40 -0700 Subject: [PATCH 45/49] Empty commit retrigger flaky tests From 2703dbf641d923a97c7c507c6e99220325a5d4e0 Mon Sep 17 00:00:00 2001 From: egalpin Date: Thu, 17 Oct 2024 15:29:30 -0700 Subject: [PATCH 46/49] Empty commit retrigger flaky tests again From 23a980fdbc2c62db4c700008e22a9c342b5daa6d Mon Sep 17 00:00:00 2001 From: egalpin Date: Fri, 18 Oct 2024 11:42:18 -0700 Subject: [PATCH 47/49] Uses bitmask for canonical requestId --- .../pinot/common/utils/request/BrokerRequestIdUtils.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/pinot-common/src/main/java/org/apache/pinot/common/utils/request/BrokerRequestIdUtils.java b/pinot-common/src/main/java/org/apache/pinot/common/utils/request/BrokerRequestIdUtils.java index a01a2a8798d..f0937469c4c 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/utils/request/BrokerRequestIdUtils.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/utils/request/BrokerRequestIdUtils.java @@ -20,6 +20,7 @@ public class BrokerRequestIdUtils { public static final int TABLE_TYPE_OFFSET = 10; + private static final long CANONICAL_MASK = Long.MAX_VALUE / TABLE_TYPE_OFFSET * TABLE_TYPE_OFFSET; private BrokerRequestIdUtils() { } @@ -33,6 +34,6 @@ public static long getRealtimeRequestId(long requestId) { } public static long getCanonicalRequestId(long requestId) { - return requestId / TABLE_TYPE_OFFSET * TABLE_TYPE_OFFSET; + return requestId & CANONICAL_MASK; } } From b022cc6792e49c6cb6dcb109f744be6be46c8dc2 Mon Sep 17 00:00:00 2001 From: egalpin Date: Fri, 18 Oct 2024 13:35:13 -0700 Subject: [PATCH 48/49] Empty commit for tests From c660fcbfff91ad3b8fddcd6ef9769776a121ef4a Mon Sep 17 00:00:00 2001 From: egalpin Date: Fri, 18 Oct 2024 14:57:42 -0700 Subject: [PATCH 49/49] Empty commit for tests off-hours