Jackie-Jiang commented on a change in pull request #7838: URL: https://github.com/apache/pinot/pull/7838#discussion_r763519397
########## File path: pinot-core/src/main/java/org/apache/pinot/core/query/reduce/BaseReduceService.java ########## @@ -0,0 +1,308 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pinot.core.query.reduce; + +import com.google.common.util.concurrent.ThreadFactoryBuilder; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.ThreadFactory; +import java.util.concurrent.TimeUnit; +import javax.annotation.concurrent.ThreadSafe; +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.request.context.ExpressionContext; +import org.apache.pinot.common.response.broker.BrokerResponseNative; +import org.apache.pinot.common.response.broker.QueryProcessingException; +import org.apache.pinot.common.response.broker.ResultTable; +import org.apache.pinot.common.utils.DataTable; +import org.apache.pinot.common.utils.DataTable.MetadataKey; +import org.apache.pinot.core.query.request.context.QueryContext; +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; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/** + * This is the base reduce service. + */ +@ThreadSafe +public abstract class BaseReduceService { + + // Set the reducer priority higher than NORM but lower than MAX, because if a query is complete + // we want to deserialize and return response as soon. This is the same as server side 'pqr' threads. + protected static final int QUERY_RUNNER_THREAD_PRIORITY = 7; + // brw -> Shorthand for broker reduce worker threads. + protected static final String REDUCE_THREAD_NAME_FORMAT = "brw-%d"; + + private static final Logger LOGGER = LoggerFactory.getLogger(BaseReduceService.class); + + protected final ExecutorService _reduceExecutorService; + protected final int _maxReduceThreadsPerQuery; + protected final int _groupByTrimThreshold; + + public BaseReduceService(PinotConfiguration config) { + _maxReduceThreadsPerQuery = config.getProperty(CommonConstants.Broker.CONFIG_OF_MAX_REDUCE_THREADS_PER_QUERY, + CommonConstants.Broker.DEFAULT_MAX_REDUCE_THREADS_PER_QUERY); + _groupByTrimThreshold = config.getProperty(CommonConstants.Broker.CONFIG_OF_BROKER_GROUPBY_TRIM_THRESHOLD, + CommonConstants.Broker.DEFAULT_BROKER_GROUPBY_TRIM_THRESHOLD); + + int numThreadsInExecutorService = Runtime.getRuntime().availableProcessors(); + LOGGER.info("Initializing BrokerReduceService with {} threads, and {} max reduce threads.", + numThreadsInExecutorService, _maxReduceThreadsPerQuery); + + ThreadFactory reduceThreadFactory = + new ThreadFactoryBuilder().setDaemon(false).setPriority(QUERY_RUNNER_THREAD_PRIORITY) + .setNameFormat(REDUCE_THREAD_NAME_FORMAT).build(); + + // ExecutorService is initialized with numThreads same as availableProcessors. + _reduceExecutorService = Executors.newFixedThreadPool(numThreadsInExecutorService, reduceThreadFactory); + } + + protected static void updateAlias(QueryContext queryContext, BrokerResponseNative brokerResponseNative) { + ResultTable resultTable = brokerResponseNative.getResultTable(); + if (resultTable == null) { + return; + } + List<String> aliasList = queryContext.getAliasList(); + if (aliasList.isEmpty()) { + return; + } + + String[] columnNames = resultTable.getDataSchema().getColumnNames(); + List<ExpressionContext> selectExpressions = getSelectExpressions(queryContext.getSelectExpressions()); + int numSelectExpressions = selectExpressions.size(); + // For query like `SELECT *`, we skip alias update. + if (columnNames.length != numSelectExpressions) { + return; + } + for (int i = 0; i < numSelectExpressions; i++) { + String alias = aliasList.get(i); + if (alias != null) { + columnNames[i] = alias; + } + } + } + + protected static List<ExpressionContext> getSelectExpressions(List<ExpressionContext> selectExpressions) { + // NOTE: For DISTINCT queries, need to extract the arguments as the SELECT expressions + if (selectExpressions.size() == 1 && selectExpressions.get(0).getType() == ExpressionContext.Type.FUNCTION + && selectExpressions.get(0).getFunction().getFunctionName().equals("distinct")) { + return selectExpressions.get(0).getFunction().getArguments(); + } + return selectExpressions; + } + + protected static void updateBrokerMetrics(BrokerResponseNative brokerResponseNative, Map<String, String> metadata) { + + } + + protected void shutDown() { + _reduceExecutorService.shutdownNow(); + } + + protected static class BrokerMetricsAggregator { + List<QueryProcessingException> _processingExceptions = new ArrayList<>(); + Map<String, String> _traceInfo = new HashMap<>(); + long _numDocsScanned = 0L; + long _numEntriesScannedInFilter = 0L; + long _numEntriesScannedPostFilter = 0L; + long _numSegmentsQueried = 0L; + long _numSegmentsProcessed = 0L; + long _numSegmentsMatched = 0L; + long _numConsumingSegmentsProcessed = 0L; + long _minConsumingFreshnessTimeMs = Long.MAX_VALUE; + long _numTotalDocs = 0L; + long _offlineThreadCpuTimeNs = 0L; + long _realtimeThreadCpuTimeNs = 0L; + long _offlineSystemActivitiesCpuTimeNs = 0L; + long _realtimeSystemActivitiesCpuTimeNs = 0L; + long _offlineResponseSerializationCpuTimeNs = 0L; + long _realtimeResponseSerializationCpuTimeNs = 0L; + long _offlineTotalCpuTimeNs = 0L; + long _realtimeTotalCpuTimeNs = 0L; + boolean _numGroupsLimitReached = false; + boolean _enableTrace; + + protected BrokerMetricsAggregator(boolean enableTrace) { + _enableTrace = enableTrace; + } + + protected void addMetrics(ServerRoutingInstance routingInstance, DataTable dataTable) { Review comment: This needs to be thread safe, and suggest renaming it ```suggestion protected synchronized void aggregate(ServerRoutingInstance routingInstance, DataTable dataTable) { ``` ########## File path: pinot-core/src/main/java/org/apache/pinot/core/query/reduce/BaseReduceService.java ########## @@ -0,0 +1,308 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pinot.core.query.reduce; + +import com.google.common.util.concurrent.ThreadFactoryBuilder; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.ThreadFactory; +import java.util.concurrent.TimeUnit; +import javax.annotation.concurrent.ThreadSafe; +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.request.context.ExpressionContext; +import org.apache.pinot.common.response.broker.BrokerResponseNative; +import org.apache.pinot.common.response.broker.QueryProcessingException; +import org.apache.pinot.common.response.broker.ResultTable; +import org.apache.pinot.common.utils.DataTable; +import org.apache.pinot.common.utils.DataTable.MetadataKey; +import org.apache.pinot.core.query.request.context.QueryContext; +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; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/** + * This is the base reduce service. + */ +@ThreadSafe +public abstract class BaseReduceService { + + // Set the reducer priority higher than NORM but lower than MAX, because if a query is complete + // we want to deserialize and return response as soon. This is the same as server side 'pqr' threads. + protected static final int QUERY_RUNNER_THREAD_PRIORITY = 7; + // brw -> Shorthand for broker reduce worker threads. + protected static final String REDUCE_THREAD_NAME_FORMAT = "brw-%d"; + + private static final Logger LOGGER = LoggerFactory.getLogger(BaseReduceService.class); + + protected final ExecutorService _reduceExecutorService; + protected final int _maxReduceThreadsPerQuery; + protected final int _groupByTrimThreshold; + + public BaseReduceService(PinotConfiguration config) { + _maxReduceThreadsPerQuery = config.getProperty(CommonConstants.Broker.CONFIG_OF_MAX_REDUCE_THREADS_PER_QUERY, + CommonConstants.Broker.DEFAULT_MAX_REDUCE_THREADS_PER_QUERY); + _groupByTrimThreshold = config.getProperty(CommonConstants.Broker.CONFIG_OF_BROKER_GROUPBY_TRIM_THRESHOLD, + CommonConstants.Broker.DEFAULT_BROKER_GROUPBY_TRIM_THRESHOLD); + + int numThreadsInExecutorService = Runtime.getRuntime().availableProcessors(); + LOGGER.info("Initializing BrokerReduceService with {} threads, and {} max reduce threads.", + numThreadsInExecutorService, _maxReduceThreadsPerQuery); + + ThreadFactory reduceThreadFactory = + new ThreadFactoryBuilder().setDaemon(false).setPriority(QUERY_RUNNER_THREAD_PRIORITY) + .setNameFormat(REDUCE_THREAD_NAME_FORMAT).build(); + + // ExecutorService is initialized with numThreads same as availableProcessors. + _reduceExecutorService = Executors.newFixedThreadPool(numThreadsInExecutorService, reduceThreadFactory); + } + + protected static void updateAlias(QueryContext queryContext, BrokerResponseNative brokerResponseNative) { + ResultTable resultTable = brokerResponseNative.getResultTable(); + if (resultTable == null) { + return; + } + List<String> aliasList = queryContext.getAliasList(); + if (aliasList.isEmpty()) { + return; + } + + String[] columnNames = resultTable.getDataSchema().getColumnNames(); + List<ExpressionContext> selectExpressions = getSelectExpressions(queryContext.getSelectExpressions()); + int numSelectExpressions = selectExpressions.size(); + // For query like `SELECT *`, we skip alias update. + if (columnNames.length != numSelectExpressions) { + return; + } + for (int i = 0; i < numSelectExpressions; i++) { + String alias = aliasList.get(i); + if (alias != null) { + columnNames[i] = alias; + } + } + } + + protected static List<ExpressionContext> getSelectExpressions(List<ExpressionContext> selectExpressions) { + // NOTE: For DISTINCT queries, need to extract the arguments as the SELECT expressions + if (selectExpressions.size() == 1 && selectExpressions.get(0).getType() == ExpressionContext.Type.FUNCTION + && selectExpressions.get(0).getFunction().getFunctionName().equals("distinct")) { + return selectExpressions.get(0).getFunction().getArguments(); + } + return selectExpressions; + } + + protected static void updateBrokerMetrics(BrokerResponseNative brokerResponseNative, Map<String, String> metadata) { + + } + + protected void shutDown() { + _reduceExecutorService.shutdownNow(); + } + + protected static class BrokerMetricsAggregator { + List<QueryProcessingException> _processingExceptions = new ArrayList<>(); Review comment: `_processingExceptions`, `_traceInfo`, `_enableTrace` can be final (optional) Should we make everything `final`? ########## File path: pinot-core/src/main/java/org/apache/pinot/core/query/reduce/BaseReduceService.java ########## @@ -0,0 +1,308 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pinot.core.query.reduce; + +import com.google.common.util.concurrent.ThreadFactoryBuilder; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.ThreadFactory; +import java.util.concurrent.TimeUnit; +import javax.annotation.concurrent.ThreadSafe; +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.request.context.ExpressionContext; +import org.apache.pinot.common.response.broker.BrokerResponseNative; +import org.apache.pinot.common.response.broker.QueryProcessingException; +import org.apache.pinot.common.response.broker.ResultTable; +import org.apache.pinot.common.utils.DataTable; +import org.apache.pinot.common.utils.DataTable.MetadataKey; +import org.apache.pinot.core.query.request.context.QueryContext; +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; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/** + * This is the base reduce service. + */ +@ThreadSafe +public abstract class BaseReduceService { + + // Set the reducer priority higher than NORM but lower than MAX, because if a query is complete + // we want to deserialize and return response as soon. This is the same as server side 'pqr' threads. + protected static final int QUERY_RUNNER_THREAD_PRIORITY = 7; + // brw -> Shorthand for broker reduce worker threads. + protected static final String REDUCE_THREAD_NAME_FORMAT = "brw-%d"; + + private static final Logger LOGGER = LoggerFactory.getLogger(BaseReduceService.class); + + protected final ExecutorService _reduceExecutorService; + protected final int _maxReduceThreadsPerQuery; + protected final int _groupByTrimThreshold; + + public BaseReduceService(PinotConfiguration config) { + _maxReduceThreadsPerQuery = config.getProperty(CommonConstants.Broker.CONFIG_OF_MAX_REDUCE_THREADS_PER_QUERY, + CommonConstants.Broker.DEFAULT_MAX_REDUCE_THREADS_PER_QUERY); + _groupByTrimThreshold = config.getProperty(CommonConstants.Broker.CONFIG_OF_BROKER_GROUPBY_TRIM_THRESHOLD, + CommonConstants.Broker.DEFAULT_BROKER_GROUPBY_TRIM_THRESHOLD); + + int numThreadsInExecutorService = Runtime.getRuntime().availableProcessors(); + LOGGER.info("Initializing BrokerReduceService with {} threads, and {} max reduce threads.", + numThreadsInExecutorService, _maxReduceThreadsPerQuery); + + ThreadFactory reduceThreadFactory = + new ThreadFactoryBuilder().setDaemon(false).setPriority(QUERY_RUNNER_THREAD_PRIORITY) + .setNameFormat(REDUCE_THREAD_NAME_FORMAT).build(); + + // ExecutorService is initialized with numThreads same as availableProcessors. + _reduceExecutorService = Executors.newFixedThreadPool(numThreadsInExecutorService, reduceThreadFactory); + } + + protected static void updateAlias(QueryContext queryContext, BrokerResponseNative brokerResponseNative) { + ResultTable resultTable = brokerResponseNative.getResultTable(); + if (resultTable == null) { + return; + } + List<String> aliasList = queryContext.getAliasList(); + if (aliasList.isEmpty()) { + return; + } + + String[] columnNames = resultTable.getDataSchema().getColumnNames(); + List<ExpressionContext> selectExpressions = getSelectExpressions(queryContext.getSelectExpressions()); + int numSelectExpressions = selectExpressions.size(); + // For query like `SELECT *`, we skip alias update. + if (columnNames.length != numSelectExpressions) { + return; + } + for (int i = 0; i < numSelectExpressions; i++) { + String alias = aliasList.get(i); + if (alias != null) { + columnNames[i] = alias; + } + } + } + + protected static List<ExpressionContext> getSelectExpressions(List<ExpressionContext> selectExpressions) { + // NOTE: For DISTINCT queries, need to extract the arguments as the SELECT expressions + if (selectExpressions.size() == 1 && selectExpressions.get(0).getType() == ExpressionContext.Type.FUNCTION + && selectExpressions.get(0).getFunction().getFunctionName().equals("distinct")) { + return selectExpressions.get(0).getFunction().getArguments(); + } + return selectExpressions; + } + + protected static void updateBrokerMetrics(BrokerResponseNative brokerResponseNative, Map<String, String> metadata) { + + } + + protected void shutDown() { + _reduceExecutorService.shutdownNow(); + } + + protected static class BrokerMetricsAggregator { + List<QueryProcessingException> _processingExceptions = new ArrayList<>(); + Map<String, String> _traceInfo = new HashMap<>(); + long _numDocsScanned = 0L; + long _numEntriesScannedInFilter = 0L; + long _numEntriesScannedPostFilter = 0L; + long _numSegmentsQueried = 0L; + long _numSegmentsProcessed = 0L; + long _numSegmentsMatched = 0L; + long _numConsumingSegmentsProcessed = 0L; + long _minConsumingFreshnessTimeMs = Long.MAX_VALUE; + long _numTotalDocs = 0L; + long _offlineThreadCpuTimeNs = 0L; + long _realtimeThreadCpuTimeNs = 0L; + long _offlineSystemActivitiesCpuTimeNs = 0L; + long _realtimeSystemActivitiesCpuTimeNs = 0L; + long _offlineResponseSerializationCpuTimeNs = 0L; + long _realtimeResponseSerializationCpuTimeNs = 0L; + long _offlineTotalCpuTimeNs = 0L; + long _realtimeTotalCpuTimeNs = 0L; + boolean _numGroupsLimitReached = false; + boolean _enableTrace; + + protected BrokerMetricsAggregator(boolean enableTrace) { + _enableTrace = enableTrace; + } + + protected void addMetrics(ServerRoutingInstance routingInstance, DataTable dataTable) { + Map<String, String> metadata = dataTable.getMetadata(); + // Reduce on trace info. + if (_enableTrace) { + _traceInfo.put(routingInstance.getHostname(), metadata.get(MetadataKey.TRACE_INFO.getName())); + } + + // Reduce on exceptions. + Map<Integer, String> exceptions = dataTable.getExceptions(); + for (int key : exceptions.keySet()) { + _processingExceptions.add(new QueryProcessingException(key, exceptions.get(key))); + } + + // Reduce on execution statistics. + String numDocsScannedString = metadata.get(MetadataKey.NUM_DOCS_SCANNED.getName()); + if (numDocsScannedString != null) { + _numDocsScanned += Long.parseLong(numDocsScannedString); + } + String numEntriesScannedInFilterString = metadata.get(MetadataKey.NUM_ENTRIES_SCANNED_IN_FILTER.getName()); + if (numEntriesScannedInFilterString != null) { + _numEntriesScannedInFilter += Long.parseLong(numEntriesScannedInFilterString); + } + String numEntriesScannedPostFilterString = metadata.get(MetadataKey.NUM_ENTRIES_SCANNED_POST_FILTER.getName()); + if (numEntriesScannedPostFilterString != null) { + _numEntriesScannedPostFilter += Long.parseLong(numEntriesScannedPostFilterString); + } + String numSegmentsQueriedString = metadata.get(MetadataKey.NUM_SEGMENTS_QUERIED.getName()); + if (numSegmentsQueriedString != null) { + _numSegmentsQueried += Long.parseLong(numSegmentsQueriedString); + } + + String numSegmentsProcessedString = metadata.get(MetadataKey.NUM_SEGMENTS_PROCESSED.getName()); + if (numSegmentsProcessedString != null) { + _numSegmentsProcessed += Long.parseLong(numSegmentsProcessedString); + } + String numSegmentsMatchedString = metadata.get(MetadataKey.NUM_SEGMENTS_MATCHED.getName()); + if (numSegmentsMatchedString != null) { + _numSegmentsMatched += Long.parseLong(numSegmentsMatchedString); + } + + String numConsumingString = metadata.get(MetadataKey.NUM_CONSUMING_SEGMENTS_PROCESSED.getName()); + if (numConsumingString != null) { + _numConsumingSegmentsProcessed += Long.parseLong(numConsumingString); + } + + String minConsumingFreshnessTimeMsString = metadata.get(MetadataKey.MIN_CONSUMING_FRESHNESS_TIME_MS.getName()); + if (minConsumingFreshnessTimeMsString != null) { + _minConsumingFreshnessTimeMs = + Math.min(Long.parseLong(minConsumingFreshnessTimeMsString), _minConsumingFreshnessTimeMs); + } + + String threadCpuTimeNsString = metadata.get(MetadataKey.THREAD_CPU_TIME_NS.getName()); + if (threadCpuTimeNsString != null) { + if (routingInstance.getTableType() == TableType.OFFLINE) { + _offlineThreadCpuTimeNs += Long.parseLong(threadCpuTimeNsString); + } else { + _realtimeThreadCpuTimeNs += Long.parseLong(threadCpuTimeNsString); + } + } + + String systemActivitiesCpuTimeNsString = metadata.get(MetadataKey.SYSTEM_ACTIVITIES_CPU_TIME_NS.getName()); + if (systemActivitiesCpuTimeNsString != null) { + if (routingInstance.getTableType() == TableType.OFFLINE) { + _offlineSystemActivitiesCpuTimeNs += Long.parseLong(systemActivitiesCpuTimeNsString); + } else { + _realtimeSystemActivitiesCpuTimeNs += Long.parseLong(systemActivitiesCpuTimeNsString); + } + } + + String responseSerializationCpuTimeNsString = metadata.get(MetadataKey.RESPONSE_SER_CPU_TIME_NS.getName()); + if (responseSerializationCpuTimeNsString != null) { + if (routingInstance.getTableType() == TableType.OFFLINE) { + _offlineResponseSerializationCpuTimeNs += Long.parseLong(responseSerializationCpuTimeNsString); + } else { + _realtimeResponseSerializationCpuTimeNs += Long.parseLong(responseSerializationCpuTimeNsString); + } + } + _offlineTotalCpuTimeNs = + _offlineThreadCpuTimeNs + _offlineSystemActivitiesCpuTimeNs + _offlineResponseSerializationCpuTimeNs; + _realtimeTotalCpuTimeNs = + _realtimeThreadCpuTimeNs + _realtimeSystemActivitiesCpuTimeNs + _realtimeResponseSerializationCpuTimeNs; + + String numTotalDocsString = metadata.get(MetadataKey.TOTAL_DOCS.getName()); + if (numTotalDocsString != null) { + _numTotalDocs += Long.parseLong(numTotalDocsString); + } + _numGroupsLimitReached |= Boolean.parseBoolean(metadata.get(MetadataKey.NUM_GROUPS_LIMIT_REACHED.getName())); + } + + protected void setBrokerMetrics(String rawTableName, BrokerResponseNative brokerResponseNative, Review comment: Suggest renaming to `setMetadata` ########## File path: pinot-core/src/main/java/org/apache/pinot/core/query/reduce/BaseReduceService.java ########## @@ -0,0 +1,308 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pinot.core.query.reduce; + +import com.google.common.util.concurrent.ThreadFactoryBuilder; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.ThreadFactory; +import java.util.concurrent.TimeUnit; +import javax.annotation.concurrent.ThreadSafe; +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.request.context.ExpressionContext; +import org.apache.pinot.common.response.broker.BrokerResponseNative; +import org.apache.pinot.common.response.broker.QueryProcessingException; +import org.apache.pinot.common.response.broker.ResultTable; +import org.apache.pinot.common.utils.DataTable; +import org.apache.pinot.common.utils.DataTable.MetadataKey; +import org.apache.pinot.core.query.request.context.QueryContext; +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; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/** + * This is the base reduce service. + */ +@ThreadSafe +public abstract class BaseReduceService { + + // Set the reducer priority higher than NORM but lower than MAX, because if a query is complete + // we want to deserialize and return response as soon. This is the same as server side 'pqr' threads. + protected static final int QUERY_RUNNER_THREAD_PRIORITY = 7; + // brw -> Shorthand for broker reduce worker threads. + protected static final String REDUCE_THREAD_NAME_FORMAT = "brw-%d"; + + private static final Logger LOGGER = LoggerFactory.getLogger(BaseReduceService.class); + + protected final ExecutorService _reduceExecutorService; + protected final int _maxReduceThreadsPerQuery; + protected final int _groupByTrimThreshold; + + public BaseReduceService(PinotConfiguration config) { + _maxReduceThreadsPerQuery = config.getProperty(CommonConstants.Broker.CONFIG_OF_MAX_REDUCE_THREADS_PER_QUERY, + CommonConstants.Broker.DEFAULT_MAX_REDUCE_THREADS_PER_QUERY); + _groupByTrimThreshold = config.getProperty(CommonConstants.Broker.CONFIG_OF_BROKER_GROUPBY_TRIM_THRESHOLD, + CommonConstants.Broker.DEFAULT_BROKER_GROUPBY_TRIM_THRESHOLD); + + int numThreadsInExecutorService = Runtime.getRuntime().availableProcessors(); + LOGGER.info("Initializing BrokerReduceService with {} threads, and {} max reduce threads.", + numThreadsInExecutorService, _maxReduceThreadsPerQuery); + + ThreadFactory reduceThreadFactory = + new ThreadFactoryBuilder().setDaemon(false).setPriority(QUERY_RUNNER_THREAD_PRIORITY) + .setNameFormat(REDUCE_THREAD_NAME_FORMAT).build(); + + // ExecutorService is initialized with numThreads same as availableProcessors. + _reduceExecutorService = Executors.newFixedThreadPool(numThreadsInExecutorService, reduceThreadFactory); + } + + protected static void updateAlias(QueryContext queryContext, BrokerResponseNative brokerResponseNative) { + ResultTable resultTable = brokerResponseNative.getResultTable(); + if (resultTable == null) { + return; + } + List<String> aliasList = queryContext.getAliasList(); + if (aliasList.isEmpty()) { + return; + } + + String[] columnNames = resultTable.getDataSchema().getColumnNames(); + List<ExpressionContext> selectExpressions = getSelectExpressions(queryContext.getSelectExpressions()); + int numSelectExpressions = selectExpressions.size(); + // For query like `SELECT *`, we skip alias update. + if (columnNames.length != numSelectExpressions) { + return; + } + for (int i = 0; i < numSelectExpressions; i++) { + String alias = aliasList.get(i); + if (alias != null) { + columnNames[i] = alias; + } + } + } + + protected static List<ExpressionContext> getSelectExpressions(List<ExpressionContext> selectExpressions) { + // NOTE: For DISTINCT queries, need to extract the arguments as the SELECT expressions + if (selectExpressions.size() == 1 && selectExpressions.get(0).getType() == ExpressionContext.Type.FUNCTION + && selectExpressions.get(0).getFunction().getFunctionName().equals("distinct")) { + return selectExpressions.get(0).getFunction().getArguments(); + } + return selectExpressions; + } + + protected static void updateBrokerMetrics(BrokerResponseNative brokerResponseNative, Map<String, String> metadata) { + + } + + protected void shutDown() { + _reduceExecutorService.shutdownNow(); + } + + protected static class BrokerMetricsAggregator { Review comment: Suggest renaming it to `QueryMetadataAggregator` ########## File path: pinot-core/src/main/java/org/apache/pinot/core/query/reduce/BaseReduceService.java ########## @@ -0,0 +1,308 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pinot.core.query.reduce; + +import com.google.common.util.concurrent.ThreadFactoryBuilder; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.ThreadFactory; +import java.util.concurrent.TimeUnit; +import javax.annotation.concurrent.ThreadSafe; +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.request.context.ExpressionContext; +import org.apache.pinot.common.response.broker.BrokerResponseNative; +import org.apache.pinot.common.response.broker.QueryProcessingException; +import org.apache.pinot.common.response.broker.ResultTable; +import org.apache.pinot.common.utils.DataTable; +import org.apache.pinot.common.utils.DataTable.MetadataKey; +import org.apache.pinot.core.query.request.context.QueryContext; +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; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/** + * This is the base reduce service. + */ +@ThreadSafe +public abstract class BaseReduceService { + + // Set the reducer priority higher than NORM but lower than MAX, because if a query is complete + // we want to deserialize and return response as soon. This is the same as server side 'pqr' threads. + protected static final int QUERY_RUNNER_THREAD_PRIORITY = 7; + // brw -> Shorthand for broker reduce worker threads. + protected static final String REDUCE_THREAD_NAME_FORMAT = "brw-%d"; + + private static final Logger LOGGER = LoggerFactory.getLogger(BaseReduceService.class); + + protected final ExecutorService _reduceExecutorService; + protected final int _maxReduceThreadsPerQuery; + protected final int _groupByTrimThreshold; + + public BaseReduceService(PinotConfiguration config) { + _maxReduceThreadsPerQuery = config.getProperty(CommonConstants.Broker.CONFIG_OF_MAX_REDUCE_THREADS_PER_QUERY, + CommonConstants.Broker.DEFAULT_MAX_REDUCE_THREADS_PER_QUERY); + _groupByTrimThreshold = config.getProperty(CommonConstants.Broker.CONFIG_OF_BROKER_GROUPBY_TRIM_THRESHOLD, + CommonConstants.Broker.DEFAULT_BROKER_GROUPBY_TRIM_THRESHOLD); + + int numThreadsInExecutorService = Runtime.getRuntime().availableProcessors(); + LOGGER.info("Initializing BrokerReduceService with {} threads, and {} max reduce threads.", + numThreadsInExecutorService, _maxReduceThreadsPerQuery); + + ThreadFactory reduceThreadFactory = + new ThreadFactoryBuilder().setDaemon(false).setPriority(QUERY_RUNNER_THREAD_PRIORITY) + .setNameFormat(REDUCE_THREAD_NAME_FORMAT).build(); + + // ExecutorService is initialized with numThreads same as availableProcessors. + _reduceExecutorService = Executors.newFixedThreadPool(numThreadsInExecutorService, reduceThreadFactory); + } + + protected static void updateAlias(QueryContext queryContext, BrokerResponseNative brokerResponseNative) { + ResultTable resultTable = brokerResponseNative.getResultTable(); + if (resultTable == null) { + return; + } + List<String> aliasList = queryContext.getAliasList(); + if (aliasList.isEmpty()) { + return; + } + + String[] columnNames = resultTable.getDataSchema().getColumnNames(); + List<ExpressionContext> selectExpressions = getSelectExpressions(queryContext.getSelectExpressions()); + int numSelectExpressions = selectExpressions.size(); + // For query like `SELECT *`, we skip alias update. + if (columnNames.length != numSelectExpressions) { + return; + } + for (int i = 0; i < numSelectExpressions; i++) { + String alias = aliasList.get(i); + if (alias != null) { + columnNames[i] = alias; + } + } + } + + protected static List<ExpressionContext> getSelectExpressions(List<ExpressionContext> selectExpressions) { + // NOTE: For DISTINCT queries, need to extract the arguments as the SELECT expressions + if (selectExpressions.size() == 1 && selectExpressions.get(0).getType() == ExpressionContext.Type.FUNCTION + && selectExpressions.get(0).getFunction().getFunctionName().equals("distinct")) { + return selectExpressions.get(0).getFunction().getArguments(); + } + return selectExpressions; + } + + protected static void updateBrokerMetrics(BrokerResponseNative brokerResponseNative, Map<String, String> metadata) { Review comment: Remove this unused method ########## File path: pinot-core/src/main/java/org/apache/pinot/core/query/reduce/SelectionOnlyStreamingReducer.java ########## @@ -0,0 +1,98 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pinot.core.query.reduce; + +import com.google.common.base.Preconditions; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import org.apache.pinot.common.response.broker.BrokerResponseNative; +import org.apache.pinot.common.response.broker.ResultTable; +import org.apache.pinot.common.utils.DataSchema; +import org.apache.pinot.common.utils.DataTable; +import org.apache.pinot.core.query.request.context.QueryContext; +import org.apache.pinot.core.query.selection.SelectionOperatorUtils; +import org.apache.pinot.core.transport.ServerRoutingInstance; +import org.apache.pinot.core.util.QueryOptionsUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +public class SelectionOnlyStreamingReducer implements StreamingReducer { + private static final Logger LOGGER = LoggerFactory.getLogger(SelectionOnlyStreamingReducer.class); + + private final QueryContext _queryContext; + private final boolean _preserveType; + private final int _limit; + + private DataSchema _dataSchema; + private DataTableReducerContext _dataTableReducerContext; + private List<Object[]> _rows; + + public SelectionOnlyStreamingReducer(QueryContext queryContext) { + _queryContext = queryContext; + _limit = _queryContext.getLimit(); + Map<String, String> queryOptions = queryContext.getQueryOptions(); + Preconditions.checkState(QueryOptionsUtils.isResponseFormatSQL(queryOptions), "only SQL response is supported"); + + _preserveType = QueryOptionsUtils.isPreserveType(queryOptions); + _dataSchema = null; + } + + @Override + public void init(DataTableReducerContext dataTableReducerContext) { + _dataTableReducerContext = dataTableReducerContext; + _rows = new ArrayList<>(Math.min(_limit, SelectionOperatorUtils.MAX_ROW_HOLDER_INITIAL_CAPACITY)); + } + + @Override + public void reduce(ServerRoutingInstance key, DataTable dataTable) { Review comment: This needs to be thread safe ########## File path: pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/GrpcBrokerRequestHandler.java ########## @@ -0,0 +1,162 @@ +/** + * 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 java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicInteger; +import javax.annotation.Nullable; +import javax.annotation.concurrent.ThreadSafe; +import org.apache.pinot.broker.api.RequestStatistics; +import org.apache.pinot.broker.broker.AccessControlFactory; +import org.apache.pinot.broker.queryquota.QueryQuotaManager; +import org.apache.pinot.broker.routing.RoutingManager; +import org.apache.pinot.common.metrics.BrokerMetrics; +import org.apache.pinot.common.proto.Server; +import org.apache.pinot.common.request.BrokerRequest; +import org.apache.pinot.common.response.broker.BrokerResponseNative; +import org.apache.pinot.common.utils.grpc.GrpcQueryClient; +import org.apache.pinot.common.utils.grpc.GrpcRequestBuilder; +import org.apache.pinot.common.utils.helix.TableCache; +import org.apache.pinot.core.query.reduce.StreamingReduceService; +import org.apache.pinot.core.transport.ServerInstance; +import org.apache.pinot.core.transport.ServerRoutingInstance; +import org.apache.pinot.core.transport.TlsConfig; +import org.apache.pinot.spi.config.table.TableType; +import org.apache.pinot.spi.env.PinotConfiguration; +import org.apache.pinot.spi.utils.builder.TableNameBuilder; + + +/** + * The <code>GrpcBrokerRequestHandler</code> class communicates query request via GRPC. + */ +@ThreadSafe +public class GrpcBrokerRequestHandler extends BaseBrokerRequestHandler { + + private final GrpcQueryClient.Config _grpcConfig; + private final Map<String, AtomicInteger> _concurrentQueriesCountMap = new ConcurrentHashMap<>(); + private final StreamingReduceService _streamingReduceService; + private final PinotStreamingQueryClient _streamingQueryClient; + + public GrpcBrokerRequestHandler(PinotConfiguration config, RoutingManager routingManager, + AccessControlFactory accessControlFactory, QueryQuotaManager queryQuotaManager, TableCache tableCache, + BrokerMetrics brokerMetrics, TlsConfig tlsConfig) { + super(config, routingManager, accessControlFactory, queryQuotaManager, tableCache, brokerMetrics); + _grpcConfig = buildGrpcQueryClientConfig(config); + + // create streaming query client + _streamingQueryClient = new PinotStreamingQueryClient(_grpcConfig); + + // create streaming reduce service + _streamingReduceService = new StreamingReduceService(config); + } + + @Override + public void start() { + } + + @Override + public synchronized void shutDown() { + _streamingReduceService.shutDown(); + } + + @Override + protected BrokerResponseNative processBrokerRequest(long requestId, BrokerRequest originalBrokerRequest, + @Nullable BrokerRequest offlineBrokerRequest, @Nullable Map<ServerInstance, List<String>> offlineRoutingTable, + @Nullable BrokerRequest realtimeBrokerRequest, @Nullable Map<ServerInstance, List<String>> realtimeRoutingTable, + long timeoutMs, ServerStats serverStats, RequestStatistics requestStatistics) + throws Exception { + assert offlineBrokerRequest != null || realtimeBrokerRequest != null; + + String rawTableName = TableNameBuilder.extractRawTableName(originalBrokerRequest.getQuerySource().getTableName()); + Map<ServerRoutingInstance, Iterator<Server.ServerResponse>> responseMap = new HashMap<>(); + // Making request to a streaming server response. + + if (offlineBrokerRequest != null) { + // to offline servers. + assert offlineRoutingTable != null; + streamingQueryToPinotServer(requestId, _brokerId, rawTableName, TableType.OFFLINE, responseMap, + offlineBrokerRequest, offlineRoutingTable, timeoutMs, true, 1); + } + if (realtimeBrokerRequest != null) { + // to realtime servers. + assert realtimeRoutingTable != null; + streamingQueryToPinotServer(requestId, _brokerId, rawTableName, TableType.REALTIME, responseMap, + realtimeBrokerRequest, realtimeRoutingTable, timeoutMs, true, 1); + } + BrokerResponseNative brokerResponse = _streamingReduceService.reduceOnStreamResponse( + originalBrokerRequest, responseMap, timeoutMs, _brokerMetrics); + return brokerResponse; + } + + /** + * Query pinot server for data table. + */ + public void streamingQueryToPinotServer(final long requestId, final String brokerHost, final String rawTableName, + final TableType tableType, Map<ServerRoutingInstance, Iterator<Server.ServerResponse>> responseMap, + BrokerRequest brokerRequest, Map<ServerInstance, List<String>> routingTable, long connectionTimeoutInMillis, + boolean ignoreEmptyResponses, int pinotRetryCount) { + // Retries will all hit the same server because the routing decision has already been made by the pinot broker + Map<ServerRoutingInstance, Iterator<Server.ServerResponse>> serverResponseMap = new HashMap<>(); + for (Map.Entry<ServerInstance, List<String>> routingEntry : routingTable.entrySet()) { + ServerInstance serverInstance = routingEntry.getKey(); + List<String> segments = routingEntry.getValue(); + String serverHost = serverInstance.getHostname(); + int port = serverInstance.getGrpcPort(); + // ensure concurrent request count from the same grpc server cannot exceed maximum + if (!_concurrentQueriesCountMap.containsKey(serverHost)) { Review comment: Not thread safe. Let's remove this map for now, and add it back if necessary when adding the throttling -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org For additional commands, e-mail: commits-h...@pinot.apache.org