Copilot commented on code in PR #17532:
URL: https://github.com/apache/pinot/pull/17532#discussion_r2707307061
##########
pinot-broker/src/main/java/org/apache/pinot/broker/routing/manager/BaseBrokerRoutingManager.java:
##########
@@ -820,6 +878,70 @@ private void buildRoutingInternal(String
tableNameWithType) {
LOGGER.info("Rebuilt routing for table: {}", tableNameWithType);
}
+ // Build routing entries for configured table samplers (if any).
+ // These entries use the same underlying routing components, but operate
on a deterministically sampled set of
+ // segments selected during routing build.
+ List<TableSamplerConfig> tableSamplerConfigs =
tableConfig.getTableSamplers();
+ if (tableSamplerConfigs != null && !tableSamplerConfigs.isEmpty()) {
+ Map<String, RoutingEntry> samplerRoutingEntries = new HashMap<>();
+ for (TableSamplerConfig samplerConfig : tableSamplerConfigs) {
+ if (samplerConfig == null) {
+ continue;
+ }
+ String samplerName = samplerConfig.getName();
+ String samplerType = samplerConfig.getType();
+ if (StringUtils.isBlank(samplerName) ||
StringUtils.isBlank(samplerType)) {
+ LOGGER.warn("Skipping invalid table sampler config for table: {},
samplerName: {}, samplerType: {}",
+ tableNameWithType, samplerName, samplerType);
+ continue;
+ }
+ try {
+ TableSampler sampler = TableSamplerFactory.create(samplerType);
+ sampler.init(tableNameWithType, tableConfig, samplerConfig,
_propertyStore);
+
+ SegmentPreSelector samplerSegmentPreSelector =
+ new TableSamplerSegmentPreSelector(segmentPreSelector,
sampler);
+ Set<String> samplerPreSelectedOnlineSegments =
+ samplerSegmentPreSelector.preSelect(new
HashSet<>(onlineSegments));
Review Comment:
Creating a defensive copy of `onlineSegments` via `new
HashSet<>(onlineSegments)` may be unnecessary if the preSelect method doesn't
modify the input. If the preSelect contract guarantees the input is not
mutated, pass `onlineSegments` directly to avoid the copy overhead.
```suggestion
samplerSegmentPreSelector.preSelect(onlineSegments);
```
##########
pinot-broker/src/main/java/org/apache/pinot/broker/routing/manager/BaseBrokerRoutingManager.java:
##########
@@ -1087,10 +1210,31 @@ private Map<ServerInstance, SegmentsToQuery>
getServerInstanceToSegmentsMap(Stri
@Override
public List<String> getSegments(BrokerRequest brokerRequest) {
String tableNameWithType = brokerRequest.getQuerySource().getTableName();
- RoutingEntry routingEntry = _routingEntryMap.get(tableNameWithType);
+ RoutingEntry routingEntry = getRoutingEntry(brokerRequest,
tableNameWithType);
return routingEntry != null ? routingEntry.getSegments(brokerRequest) :
null;
}
+ @Nullable
+ private RoutingEntry getRoutingEntry(BrokerRequest brokerRequest, String
tableNameWithType) {
+ String samplerName = null;
+ if (brokerRequest != null && brokerRequest.isSetPinotQuery()) {
+ org.apache.pinot.common.request.PinotQuery pinotQuery =
brokerRequest.getPinotQuery();
+ if (pinotQuery != null && pinotQuery.isSetQueryOptions()) {
+ samplerName =
pinotQuery.getQueryOptions().get(CommonConstants.Broker.Request.QueryOptionKey.TABLE_SAMPLER);
+ }
+ }
+ if (StringUtils.isNotBlank(samplerName)) {
+ Map<String, RoutingEntry> samplerEntries =
_samplerRoutingEntryMap.get(tableNameWithType);
+ if (samplerEntries != null) {
+ RoutingEntry samplerEntry = samplerEntries.get(samplerName);
+ if (samplerEntry != null) {
+ return samplerEntry;
+ }
Review Comment:
The nested null checks create unnecessary nesting. Consider using a more
streamlined approach: `Map<String, RoutingEntry> samplerEntries =
_samplerRoutingEntryMap.get(tableNameWithType); if (samplerEntries != null) {
RoutingEntry samplerEntry = samplerEntries.get(samplerName); if (samplerEntry
!= null) return samplerEntry; }`
```suggestion
RoutingEntry samplerEntry =
samplerEntries != null ? samplerEntries.get(samplerName) : null;
if (samplerEntry != null) {
return samplerEntry;
```
##########
pinot-broker/src/main/java/org/apache/pinot/broker/routing/tablesampler/NPerDaySegmentsTableSampler.java:
##########
@@ -0,0 +1,122 @@
+/**
+ * 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.routing.tablesampler;
+
+import java.time.Instant;
+import java.time.ZoneId;
+import java.time.ZonedDateTime;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.commons.collections4.MapUtils;
+import org.apache.helix.store.zk.ZkHelixPropertyStore;
+import org.apache.helix.zookeeper.datamodel.ZNRecord;
+import org.apache.pinot.common.metadata.ZKMetadataProvider;
+import org.apache.pinot.common.metadata.segment.SegmentZKMetadata;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.config.table.sampler.TableSamplerConfig;
+
+
+/**
+ * Selects up to N segments per day based on segment ZK metadata start time.
+ *
+ * <p>Config:
+ * <ul>
+ * <li>{@code properties.numSegmentsPerDay}: positive integer</li>
+ * <li>{@code properties.timezone}: optional timezone ID, defaults to {@code
UTC}</li>
+ * </ul>
+ *
+ * <p>Notes:
+ * <ul>
+ * <li>Day bucketing is derived from the segment start time (in ms)
converted to the configured timezone.</li>
+ * <li>Segments without valid start time metadata are included (not sampled)
to avoid unexpectedly dropping data.</li>
+ * </ul>
+ */
+public class NPerDaySegmentsTableSampler implements TableSampler {
+ public static final String TYPE = "nPerDay";
+ public static final String PROP_NUM_SEGMENTS_PER_DAY = "numSegmentsPerDay";
+ public static final String PROP_TIMEZONE = "timezone";
+
+ private String _tableNameWithType;
+ private ZkHelixPropertyStore<ZNRecord> _propertyStore;
+ private int _numSegmentsPerDay;
+ private ZoneId _zoneId;
+
+ @Override
+ public void init(String tableNameWithType, TableConfig tableConfig,
TableSamplerConfig samplerConfig,
+ ZkHelixPropertyStore<ZNRecord> propertyStore) {
+ _tableNameWithType = tableNameWithType;
+ _propertyStore = propertyStore;
+
+ Map<String, String> props = samplerConfig.getProperties();
+ if (MapUtils.isEmpty(props) ||
!props.containsKey(PROP_NUM_SEGMENTS_PER_DAY)) {
+ throw new IllegalArgumentException(
+ "Missing required property '" + PROP_NUM_SEGMENTS_PER_DAY + "' for
table sampler type '" + TYPE + "'");
+ }
+ _numSegmentsPerDay =
Integer.parseInt(props.get(PROP_NUM_SEGMENTS_PER_DAY));
+ if (_numSegmentsPerDay <= 0) {
+ throw new IllegalArgumentException("'" + PROP_NUM_SEGMENTS_PER_DAY + "'
must be positive");
+ }
+
+ String timezone = props.getOrDefault(PROP_TIMEZONE, "UTC");
+ _zoneId = ZoneId.of(timezone);
+ }
+
+ @Override
+ public Set<String> selectSegments(Set<String> onlineSegments) {
+ if (onlineSegments.isEmpty()) {
+ return Collections.emptySet();
+ }
+
+ Map<Long, List<String>> epochDayToSegments = new HashMap<>();
+ List<String> unknownTimeSegments = new ArrayList<>();
+
+ for (String segmentName : onlineSegments) {
+ SegmentZKMetadata zkMetadata =
+ ZKMetadataProvider.getSegmentZKMetadata(_propertyStore,
_tableNameWithType, segmentName);
+ if (zkMetadata == null) {
+ unknownTimeSegments.add(segmentName);
+ continue;
+ }
+ long startTimeMs = zkMetadata.getStartTimeMs();
+ if (startTimeMs <= 0) {
+ unknownTimeSegments.add(segmentName);
Review Comment:
Fetching segment ZK metadata individually for each segment (N calls) creates
an N+1 query pattern. Consider batch-fetching all segment metadata upfront
using `ZKMetadataProvider.getSegmentZKMetadataForTable()` or similar bulk API
to reduce ZooKeeper round trips, especially for tables with many segments.
##########
pinot-broker/src/main/java/org/apache/pinot/broker/routing/manager/BaseBrokerRoutingManager.java:
##########
@@ -1087,10 +1210,31 @@ private Map<ServerInstance, SegmentsToQuery>
getServerInstanceToSegmentsMap(Stri
@Override
public List<String> getSegments(BrokerRequest brokerRequest) {
String tableNameWithType = brokerRequest.getQuerySource().getTableName();
- RoutingEntry routingEntry = _routingEntryMap.get(tableNameWithType);
+ RoutingEntry routingEntry = getRoutingEntry(brokerRequest,
tableNameWithType);
return routingEntry != null ? routingEntry.getSegments(brokerRequest) :
null;
}
+ @Nullable
+ private RoutingEntry getRoutingEntry(BrokerRequest brokerRequest, String
tableNameWithType) {
+ String samplerName = null;
+ if (brokerRequest != null && brokerRequest.isSetPinotQuery()) {
+ org.apache.pinot.common.request.PinotQuery pinotQuery =
brokerRequest.getPinotQuery();
+ if (pinotQuery != null && pinotQuery.isSetQueryOptions()) {
Review Comment:
There are redundant null checks. If `isSetPinotQuery()` returns true,
`getPinotQuery()` should not return null. Similarly, `isSetQueryOptions()`
implies the result is non-null. Simplify by removing the redundant null checks
after the `isSet` checks.
```suggestion
if (pinotQuery.isSetQueryOptions()) {
```
--
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: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]