Jackie-Jiang commented on code in PR #17532:
URL: https://github.com/apache/pinot/pull/17532#discussion_r2776716275
##########
pinot-broker/src/main/java/org/apache/pinot/broker/routing/manager/BaseBrokerRoutingManager.java:
##########
@@ -820,6 +971,69 @@ 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()) {
Review Comment:
(minor) Use `CollectionUtils.isNotEmpty()`
##########
pinot-broker/src/main/java/org/apache/pinot/broker/routing/tablesampler/TableSampler.java:
##########
@@ -0,0 +1,46 @@
+/**
+ * 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.util.Set;
+import org.apache.helix.store.zk.ZkHelixPropertyStore;
+import org.apache.helix.zookeeper.datamodel.ZNRecord;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.config.table.sampler.TableSamplerConfig;
+
+
+/**
+ * A {@code TableSampler} deterministically selects a subset of segments from
the set of online segments for a table.
+ *
+ * <p>Selection is performed during routing table build/update so there is no
additional per-query overhead beyond
+ * selecting the pre-built routing entry.
+ */
+public interface TableSampler {
+
+ /**
+ * Initializes the sampler for a specific table and sampler config.
+ */
+ void init(String tableNameWithType, TableConfig tableConfig,
TableSamplerConfig samplerConfig,
Review Comment:
No need to pass in `tableNameWithType`. Use `TableConfig.getTableNameI()`
##########
pinot-broker/src/main/java/org/apache/pinot/broker/routing/manager/BaseBrokerRoutingManager.java:
##########
@@ -143,6 +152,9 @@ public abstract class BaseBrokerRoutingManager implements
RoutingManager, Cluste
// and OFFLINE tables. LogicalTables will also use the raw table name of the
table list underneath
private final Map<String, Object> _routingTableBuildLocks = new
ConcurrentHashMap<>();
+ // Per-table locks for sampler routing entry updates.
+ private final Map<String, Object> _routingTableSamplerBuildLocks = new
ConcurrentHashMap<>();
Review Comment:
We shouldn't need this extra lock. When updating the routing for a given
table, we should grab the routing table build lock for the table, update both
the routing entry, and also the sampler routing entries if exist
##########
pinot-broker/src/main/java/org/apache/pinot/broker/routing/manager/BaseBrokerRoutingManager.java:
##########
@@ -820,6 +971,69 @@ 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) {
+ 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 =
Review Comment:
The sampler shouldn't be modeled as a pre-selector. It should be applied
after pre-selection, where lineage is handled, or it can give wrong result.
##########
pinot-broker/src/main/java/org/apache/pinot/broker/routing/manager/BaseBrokerRoutingManager.java:
##########
@@ -820,6 +971,69 @@ 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) {
+ 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 =
+ // SegmentPreSelector contract allows mutation of the input
set. Create a defensive copy so the
+ // sampler cannot mutate the shared onlineSegments set, which
is reused by other samplers.
+ samplerSegmentPreSelector.preSelect(new
HashSet<>(onlineSegments));
Review Comment:
This should be applied to `preSelectedOnlineSegments`.
##########
pinot-broker/src/main/java/org/apache/pinot/broker/routing/manager/BaseBrokerRoutingManager.java:
##########
@@ -261,25 +355,33 @@ protected void processSegmentAssignmentChangeInternal() {
if (idealStateStat != null && externalViewStat != null) {
String tableNameWithType = cachedTableNames.get(i);
futures.add(_executorService.submit(() -> {
+ AssignmentChangeResult assignmentChangeResult;
+ RoutingEntry routingEntry;
Object tableLock = getRoutingTableBuildLock(tableNameWithType);
synchronized (tableLock) {
// The routingEntry may have been removed from the
_routingEntryMap by the time we get here in case
// one of the other functions such as 'removeRouting' was called
since taking the snapshot. Check for
// existence before proceeding. Also note that if new entries were
added since the snapshot was taken, we
// will miss processing them in this call. The buildRouting()
method tries to handle that by checking for
// changes in the IS / EV version after adding a new entry
- RoutingEntry routingEntry =
_routingEntryMap.get(tableNameWithType);
+ routingEntry = _routingEntryMap.get(tableNameWithType);
if (routingEntry == null) {
LOGGER.info("Table {} was removed while processing segment
assignment change, skipping",
tableNameWithType);
return;
}
- boolean hasISOrEVVersionChanged =
processAssignmentChangeForTable(idealStateStat.getVersion(),
- externalViewStat.getVersion(), routingEntry);
- if (hasISOrEVVersionChanged) {
+ assignmentChangeResult =
+ processAssignmentChangeForTable(idealStateStat.getVersion(),
externalViewStat.getVersion(),
+ routingEntry);
+ if (assignmentChangeResult.hasVersionChanged()) {
tablesUpdated.add(tableNameWithType);
}
}
+ // Avoid reusing the per-table lock for sampler routing update so
sampler updates can proceed in parallel.
Review Comment:
I don't follow this. Wouldn't this cause race condition? Routing entry is
not protected
--
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]