Jackie-Jiang commented on code in PR #17532:
URL: https://github.com/apache/pinot/pull/17532#discussion_r2796246624


##########
pinot-broker/src/main/java/org/apache/pinot/broker/routing/manager/BaseBrokerRoutingManager.java:
##########
@@ -1248,6 +1343,45 @@ boolean isDisabled() {
       return _disabled;
     }
 
+    private void updateSamplerInfos(IdealState idealState, ExternalView 
externalView,
+        Set<String> preSelectedOnlineSegments) {
+      if (_samplerInfos.isEmpty()) {
+        return;
+      }
+      for (Map.Entry<String, SamplerInfo> entry : _samplerInfos.entrySet()) {
+        String samplerName = entry.getKey();
+        SamplerInfo samplerInfo = entry.getValue();
+        try {
+          Set<String> samplerPreSelectedOnlineSegments =
+              
samplerInfo._tableSampler.sampleSegments(preSelectedOnlineSegments);
+          if (samplerPreSelectedOnlineSegments == null) {

Review Comment:
   (minor) Same here, check is redundant



##########
pinot-common/src/main/java/org/apache/pinot/common/utils/config/QueryOptionsUtils.java:
##########
@@ -68,6 +68,8 @@ private QueryOptionsUtils() {
           }
         }
       }
+      // Backward-compatible alias used by SQL SET syntax: SET sampler='x'
+      configResolver.put("sampler", QueryOptionKey.TABLE_SAMPLER);

Review Comment:
   Is `sampler` a standard concept in SQL? If so, should we just use it as the 
option name?



##########
pinot-broker/src/main/java/org/apache/pinot/broker/routing/manager/BaseBrokerRoutingManager.java:
##########
@@ -810,10 +820,58 @@ private void buildRoutingInternal(String 
tableNameWithType) {
       }
       segmentZkMetadataFetcher.init(idealState, externalView, 
preSelectedOnlineSegments);
 
+      // Build table sampler contexts keyed by normalized sampler name.
+      Map<String, SamplerInfo> samplerInfos = Map.of();
+      List<TableSamplerConfig> tableSamplerConfigs = 
tableConfig.getTableSamplers();
+      if (CollectionUtils.isNotEmpty(tableSamplerConfigs)) {
+        Map<String, SamplerInfo> configuredSamplerInfos =
+            new 
HashMap<>(HashUtil.getHashMapCapacity(tableSamplerConfigs.size()));

Review Comment:
   (minor)
   ```suggestion
               Maps.newHashMapWithExpectedSize(tableSamplerConfigs.size());
   ```



##########
pinot-broker/src/main/java/org/apache/pinot/broker/routing/manager/BaseBrokerRoutingManager.java:
##########
@@ -810,10 +820,58 @@ private void buildRoutingInternal(String 
tableNameWithType) {
       }
       segmentZkMetadataFetcher.init(idealState, externalView, 
preSelectedOnlineSegments);
 
+      // Build table sampler contexts keyed by normalized sampler name.
+      Map<String, SamplerInfo> samplerInfos = Map.of();
+      List<TableSamplerConfig> tableSamplerConfigs = 
tableConfig.getTableSamplers();
+      if (CollectionUtils.isNotEmpty(tableSamplerConfigs)) {
+        Map<String, SamplerInfo> configuredSamplerInfos =
+            new 
HashMap<>(HashUtil.getHashMapCapacity(tableSamplerConfigs.size()));
+        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;
+          }
+          String normalizedSamplerName = normalizeSamplerName(samplerName);
+          if (configuredSamplerInfos.containsKey(normalizedSamplerName)) {
+            LOGGER.warn("Skipping duplicate normalized table sampler name: 
'{}' for table: {}", samplerName,
+                tableNameWithType);
+            continue;
+          }
+          try {
+            TableSampler sampler = TableSamplerFactory.create(samplerType);
+            sampler.init(tableConfig, samplerConfig, _propertyStore);
+            Set<String> samplerPreSelectedOnlineSegments = 
sampler.sampleSegments(preSelectedOnlineSegments);
+            if (samplerPreSelectedOnlineSegments == null) {
+              LOGGER.warn("Sampler '{}' returned null segments for table '{}', 
treating it as empty", samplerName,
+                  tableNameWithType);
+              samplerPreSelectedOnlineSegments = Collections.emptySet();

Review Comment:
   (nit) `Set.of()`



##########
pinot-broker/src/main/java/org/apache/pinot/broker/routing/manager/BaseBrokerRoutingManager.java:
##########
@@ -1086,9 +1146,30 @@ private Map<ServerInstance, SegmentsToQuery> 
getServerInstanceToSegmentsMap(Stri
   @Nullable
   @Override
   public List<String> getSegments(BrokerRequest brokerRequest) {
+    return getSegments(brokerRequest, extractSamplerName(brokerRequest));
+  }
+
+  @Nullable
+  @Override
+  public List<String> getSegments(BrokerRequest brokerRequest, @Nullable 
String samplerName) {
     String tableNameWithType = brokerRequest.getQuerySource().getTableName();
     RoutingEntry routingEntry = _routingEntryMap.get(tableNameWithType);
-    return routingEntry != null ? routingEntry.getSegments(brokerRequest) : 
null;
+    if (routingEntry == null) {
+      return null;
+    }
+    return routingEntry.getSegments(brokerRequest, samplerName);
+  }
+
+  private static String normalizeSamplerName(String samplerName) {
+    return samplerName.trim().toLowerCase(Locale.ROOT);
+  }
+
+  @Nullable
+  static String extractSamplerName(BrokerRequest brokerRequest) {
+    if (!brokerRequest.isSetPinotQuery() || 
!brokerRequest.getPinotQuery().isSetQueryOptions()) {
+      return null;
+    }

Review Comment:
   (minor) This check shouldn't be required



##########
pinot-common/src/main/java/org/apache/pinot/common/utils/config/QueryOptionsUtils.java:
##########
@@ -111,6 +113,11 @@ public static Long getTimeoutMs(Map<String, String> 
queryOptions) {
     return checkedParseLongPositive(QueryOptionKey.TIMEOUT_MS, 
timeoutMsString);
   }
 
+  @Nullable
+  public static String getTableSampler(@Nullable Map<String, String> 
queryOptions) {

Review Comment:
   (minor) To be consistent with other methods, remove `@Nullable` for 
`queryOptions`. It is never null in `BrokerRequest`



##########
pinot-broker/src/main/java/org/apache/pinot/broker/routing/manager/BaseBrokerRoutingManager.java:
##########
@@ -810,10 +820,58 @@ private void buildRoutingInternal(String 
tableNameWithType) {
       }
       segmentZkMetadataFetcher.init(idealState, externalView, 
preSelectedOnlineSegments);
 
+      // Build table sampler contexts keyed by normalized sampler name.
+      Map<String, SamplerInfo> samplerInfos = Map.of();
+      List<TableSamplerConfig> tableSamplerConfigs = 
tableConfig.getTableSamplers();
+      if (CollectionUtils.isNotEmpty(tableSamplerConfigs)) {
+        Map<String, SamplerInfo> configuredSamplerInfos =
+            new 
HashMap<>(HashUtil.getHashMapCapacity(tableSamplerConfigs.size()));
+        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;
+          }
+          String normalizedSamplerName = normalizeSamplerName(samplerName);
+          if (configuredSamplerInfos.containsKey(normalizedSamplerName)) {
+            LOGGER.warn("Skipping duplicate normalized table sampler name: 
'{}' for table: {}", samplerName,
+                tableNameWithType);
+            continue;
+          }
+          try {
+            TableSampler sampler = TableSamplerFactory.create(samplerType);
+            sampler.init(tableConfig, samplerConfig, _propertyStore);
+            Set<String> samplerPreSelectedOnlineSegments = 
sampler.sampleSegments(preSelectedOnlineSegments);
+            if (samplerPreSelectedOnlineSegments == null) {

Review Comment:
   (minor) What is the contract for `sampleSegments()`? Do we allow it 
returning `null`?



-- 
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]

Reply via email to