You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by "walterddr (via GitHub)" <gi...@apache.org> on 2023/06/05 14:55:48 UTC

[GitHub] [pinot] walterddr opened a new pull request, #10461: add partition metadata manager

walterddr opened a new pull request, #10461:
URL: https://github.com/apache/pinot/pull/10461

   This is a follow up on #10455 
   
   Previous PR
   ===
   - previous PR #10455 adds `SegmentZkMetadataFetcher` and `SegmentZkMetadataFetcherListener`
   
   This PR
   ===
   - [x] Create PartitionDataManager which similar to SinglePartitionColumnSegmentPruner but kept a very different view of the map
       - [x] create `Map<PartitionID, List<Server>> partitionToServerMapping` to indicate whole-partition-on-a-single-server
       - [x] create `Map<PartitionID, List<Segment>> partitionToSegmentMapping` to indicate partition and segment relationship
   - [x] ~~TimeBoundaryManager to SegmentZkMetadataFetcher management~~
   
   Follow Up
   ===
   due to the complexity of TimeBoundaryManager which needs to coordinate between REALTIME and OFFLINE table loading, we will leaf this one out from SegmentZkMetadataFetcher; to summarize
     - all `SegmentPruners` and `SegmentPartitionMetadataManager` will extend `SegmentZkMetadataFetchListener` and will no longer individually pull and cache from ZK
     - `TimeBoundaryManager` will remain on its own cycle to pull and cache data from ZK


-- 
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-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] walterddr commented on a diff in pull request #10461: add partition metadata manager

Posted by "walterddr (via GitHub)" <gi...@apache.org>.
walterddr commented on code in PR #10461:
URL: https://github.com/apache/pinot/pull/10461#discussion_r1155161875


##########
pinot-broker/src/main/java/org/apache/pinot/broker/routing/segmentpartition/SegmentPartitionMetadataManager.java:
##########
@@ -0,0 +1,189 @@
+/**
+ * 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.segmentpartition;
+
+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 java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.helix.model.ExternalView;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.zookeeper.datamodel.ZNRecord;
+import org.apache.pinot.broker.routing.segmentmetadata.SegmentZkMetadataFetchListener;
+import org.apache.pinot.spi.utils.CommonConstants;
+
+
+/**
+ * The {@code PartitionDataManager} manages partitions of a table. It manages
+ *   1. all the online segments associated with the partition and their allocated servers
+ *   2. all the replica of a specific segment.
+ * It provides API to query
+ *   1. For each partition ID, what are the servers that contains ALL segments belong to this partition ID.
+ *   2. For each server, what are all the partition IDs and list of segments of those partition IDs on this server.
+ */
+public class SegmentPartitionMetadataManager implements SegmentZkMetadataFetchListener {
+  private final String _tableNameWithType;
+
+  // static content, if anything changes for the following. a rebuild of routing table is needed.
+  private final String _partitionColumn;
+  private final String _partitionFunctionName;
+  private final int _numPartitions;
+
+  // cache-able content, only follow changes if onlineSegments list (of ideal-state) is changed.
+  private final Map<String, List<String>> _segmentToOnlineServersMap = new HashMap<>();
+  private final Map<String, Integer> _segmentToPartitionMap = new HashMap<>();
+
+  // computed value based on status change.
+  private Map<Integer, Set<String>> _partitionToFullyReplicatedServersMap;
+  private Map<Integer, List<String>> _partitionToSegmentsMap;
+
+  public SegmentPartitionMetadataManager(String tableNameWithType, String partitionColumn, String partitionFunctionName,
+      int numPartitions) {
+    _tableNameWithType = tableNameWithType;
+    _partitionColumn = partitionColumn;
+    _partitionFunctionName = partitionFunctionName;
+    _numPartitions = numPartitions;
+  }
+
+  @Override
+  public void init(IdealState idealState, ExternalView externalView, List<String> onlineSegments,
+      List<ZNRecord> znRecords) {
+    for (int idx = 0; idx < onlineSegments.size(); idx++) {
+      String segment = onlineSegments.get(idx);
+      ZNRecord znRecord = znRecords.get(idx);
+      // extract single partition info
+      SegmentPartitionInfo segmentPartitionInfo = SegmentPartitionUtils.extractPartitionInfoFromSegmentZKMetadata(
+          _tableNameWithType, _partitionColumn, segment, znRecord);
+      if (validate(segmentPartitionInfo)) {
+        // update segment to partition map
+        Integer partitionId = segmentPartitionInfo.getPartitions().iterator().next();
+        _segmentToPartitionMap.put(segment, partitionId);
+      }
+      // update segment to server list.
+      updateSegmentServerOnlineMap(externalView, segment);
+    }
+    computePartitionMaps();
+  }
+
+  private void updateSegmentServerOnlineMap(ExternalView externalView, String segment) {
+    Map<String, String> instanceStateMap = externalView.getStateMap(segment);
+    List<String> serverList = instanceStateMap == null ? Collections.emptyList()
+        : instanceStateMap.entrySet().stream()
+            .filter(entry -> CommonConstants.Helix.StateModel.SegmentStateModel.ONLINE.equals(entry.getValue()))
+            .map(Map.Entry::getKey)
+            .collect(Collectors.toList());
+    _segmentToOnlineServersMap.put(segment, serverList);
+  }
+
+  private boolean validate(SegmentPartitionInfo segmentPartitionInfo) {
+    if (segmentPartitionInfo == null || segmentPartitionInfo.getPartitionFunction() == null
+        || segmentPartitionInfo.getPartitions() == null) {
+      return false;
+    }
+    // TODO: check more than just function name here but also function config.
+    return _partitionFunctionName.equals(segmentPartitionInfo.getPartitionFunction().getName())
+        && _partitionColumn.equals(segmentPartitionInfo.getPartitionColumn())
+        && _numPartitions == segmentPartitionInfo.getNumPartitions()
+        && segmentPartitionInfo.getPartitions().size() == 1;
+  }
+
+  @Override
+  public synchronized void onAssignmentChange(IdealState idealState, ExternalView externalView,
+      Set<String> onlineSegments, List<String> pulledSegments, List<ZNRecord> znRecords) {
+    // update segment zk metadata for the pulled segments
+    for (int idx = 0; idx < pulledSegments.size(); idx++) {
+      String segment = pulledSegments.get(idx);
+      ZNRecord znRecord = znRecords.get(idx);
+      SegmentPartitionInfo segmentPartitionInfo = SegmentPartitionUtils.extractPartitionInfoFromSegmentZKMetadata(
+          _tableNameWithType, _partitionColumn, segment, znRecord);
+      if (validate(segmentPartitionInfo)) {
+        // update segment to partition map
+        Integer partitionId = segmentPartitionInfo.getPartitions().iterator().next();
+        _segmentToPartitionMap.put(segment, partitionId);
+      } else {
+        // remove the segment from the partition map
+        _segmentToPartitionMap.remove(segment);
+      }
+    }
+    // update the server online information based on external view.
+    for (String onlineSegment : onlineSegments) {
+      // update segment to server list.
+      updateSegmentServerOnlineMap(externalView, onlineSegment);
+    }
+    _segmentToPartitionMap.keySet().retainAll(onlineSegments);
+    _segmentToOnlineServersMap.keySet().retainAll(onlineSegments);
+    computePartitionMaps();
+  }
+
+  @Override
+  public synchronized void refreshSegment(String segment, @Nullable ZNRecord znRecord) {

Review Comment:
   review note: when refreshSegment, IS and EV are not available. assuming no change here thus not updating the segment to server map



-- 
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-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] walterddr commented on a diff in pull request #10461: add partition metadata manager

Posted by "walterddr (via GitHub)" <gi...@apache.org>.
walterddr commented on code in PR #10461:
URL: https://github.com/apache/pinot/pull/10461#discussion_r1146892699


##########
pinot-broker/src/main/java/org/apache/pinot/broker/routing/BrokerRoutingManager.java:
##########
@@ -478,20 +486,35 @@ public synchronized void buildRouting(String tableNameWithType) {
         Set<String> offlineTablePreSelectedOnlineSegments =
             offlineTableSegmentPreSelector.preSelect(offlineTableOnlineSegments);
         TimeBoundaryManager offlineTableTimeBoundaryManager =
-            new TimeBoundaryManager(offlineTableConfig, _propertyStore, _brokerMetrics);
-        offlineTableTimeBoundaryManager.init(offlineTableIdealState, offlineTableExternalView,
-            offlineTablePreSelectedOnlineSegments);
-        offlineTableRoutingEntry.setTimeBoundaryManager(offlineTableTimeBoundaryManager);
+            new TimeBoundaryManager(offlineTableConfig, schema, _brokerMetrics);
+        offlineTableRoutingEntry.initTimeBoundaryManager(offlineTableTimeBoundaryManager, offlineTableIdealState,
+            offlineTableExternalView, offlineTablePreSelectedOnlineSegments);

Review Comment:
   This init is not protected by synchronize and thus I cannot move it to be init via SegmentZkMetadataCache...
   
   This potentially can cause out-of-sync between the cache and the TimeBoundaryManager
   
   In order for this to be called via SegmentZkMetadataCache, we can either:
   - make onAssignmentChange API call (which is synchronized);
   - make the init method also synchronized



-- 
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-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] Jackie-Jiang merged pull request #10461: add partition metadata manager

Posted by "Jackie-Jiang (via GitHub)" <gi...@apache.org>.
Jackie-Jiang merged PR #10461:
URL: https://github.com/apache/pinot/pull/10461


-- 
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-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] walterddr closed pull request #10461: add partition metadata manager

Posted by "walterddr (via GitHub)" <gi...@apache.org>.
walterddr closed pull request #10461: add partition metadata manager
URL: https://github.com/apache/pinot/pull/10461


-- 
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-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] walterddr closed pull request #10461: add partition metadata manager

Posted by "walterddr (via GitHub)" <gi...@apache.org>.
walterddr closed pull request #10461: add partition metadata manager
URL: https://github.com/apache/pinot/pull/10461


-- 
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-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] walterddr commented on a diff in pull request #10461: add partition metadata manager

Posted by "walterddr (via GitHub)" <gi...@apache.org>.
walterddr commented on code in PR #10461:
URL: https://github.com/apache/pinot/pull/10461#discussion_r1155162062


##########
pinot-broker/src/main/java/org/apache/pinot/broker/routing/segmentpartition/SegmentPartitionMetadataManager.java:
##########
@@ -0,0 +1,189 @@
+/**
+ * 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.segmentpartition;
+
+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 java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.helix.model.ExternalView;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.zookeeper.datamodel.ZNRecord;
+import org.apache.pinot.broker.routing.segmentmetadata.SegmentZkMetadataFetchListener;
+import org.apache.pinot.spi.utils.CommonConstants;
+
+
+/**
+ * The {@code PartitionDataManager} manages partitions of a table. It manages
+ *   1. all the online segments associated with the partition and their allocated servers
+ *   2. all the replica of a specific segment.
+ * It provides API to query
+ *   1. For each partition ID, what are the servers that contains ALL segments belong to this partition ID.
+ *   2. For each server, what are all the partition IDs and list of segments of those partition IDs on this server.
+ */
+public class SegmentPartitionMetadataManager implements SegmentZkMetadataFetchListener {
+  private final String _tableNameWithType;
+
+  // static content, if anything changes for the following. a rebuild of routing table is needed.
+  private final String _partitionColumn;
+  private final String _partitionFunctionName;
+  private final int _numPartitions;
+
+  // cache-able content, only follow changes if onlineSegments list (of ideal-state) is changed.
+  private final Map<String, List<String>> _segmentToOnlineServersMap = new HashMap<>();
+  private final Map<String, Integer> _segmentToPartitionMap = new HashMap<>();
+
+  // computed value based on status change.
+  private Map<Integer, Set<String>> _partitionToFullyReplicatedServersMap;
+  private Map<Integer, List<String>> _partitionToSegmentsMap;
+
+  public SegmentPartitionMetadataManager(String tableNameWithType, String partitionColumn, String partitionFunctionName,
+      int numPartitions) {
+    _tableNameWithType = tableNameWithType;
+    _partitionColumn = partitionColumn;
+    _partitionFunctionName = partitionFunctionName;
+    _numPartitions = numPartitions;
+  }
+
+  @Override
+  public void init(IdealState idealState, ExternalView externalView, List<String> onlineSegments,

Review Comment:
   review note: when `init` and `onAssignmentChange` it is actually 2-step
   1. update with zk metadata (for partition mapping) -- done only on those that have zk metadata changed
   2. update with external vew (for online server mapping) -- done on every online segment
   
   



-- 
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-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] codecov-commenter commented on pull request #10461: add partition metadata manager

Posted by "codecov-commenter (via GitHub)" <gi...@apache.org>.
codecov-commenter commented on PR #10461:
URL: https://github.com/apache/pinot/pull/10461#issuecomment-1481862042

   ## [Codecov](https://codecov.io/gh/apache/pinot/pull/10461?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#10461](https://codecov.io/gh/apache/pinot/pull/10461?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (dd8afa9) into [master](https://codecov.io/gh/apache/pinot/commit/c0d0896ac30ca04229f641b960c2722d98398de3?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (c0d0896) will **decrease** coverage by `10.52%`.
   > The diff coverage is `64.96%`.
   
   ```diff
   @@              Coverage Diff              @@
   ##             master   #10461       +/-   ##
   =============================================
   - Coverage     24.44%   13.93%   -10.52%     
   - Complexity       49      237      +188     
   =============================================
     Files          2042     2017       -25     
     Lines        111146   109573     -1573     
     Branches      16933    16749      -184     
   =============================================
   - Hits          27174    15265    -11909     
   - Misses        81132    93062    +11930     
   + Partials       2840     1246     -1594     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration1 | `?` | |
   | unittests2 | `13.93% <64.96%> (?)` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/pinot/pull/10461?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...mentpartition/SegmentPartitionMetadataManager.java](https://codecov.io/gh/apache/pinot/pull/10461?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvcm91dGluZy9zZWdtZW50cGFydGl0aW9uL1NlZ21lbnRQYXJ0aXRpb25NZXRhZGF0YU1hbmFnZXIuamF2YQ==) | `0.00% <0.00%> (ø)` | |
   | [...che/pinot/broker/routing/BrokerRoutingManager.java](https://codecov.io/gh/apache/pinot/pull/10461?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvcm91dGluZy9Ccm9rZXJSb3V0aW5nTWFuYWdlci5qYXZh) | `60.49% <53.12%> (-24.47%)` | :arrow_down: |
   | [...outing/segmentpartition/SegmentPartitionUtils.java](https://codecov.io/gh/apache/pinot/pull/10461?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvcm91dGluZy9zZWdtZW50cGFydGl0aW9uL1NlZ21lbnRQYXJ0aXRpb25VdGlscy5qYXZh) | `70.37% <70.37%> (ø)` | |
   | [...mentpruner/SinglePartitionColumnSegmentPruner.java](https://codecov.io/gh/apache/pinot/pull/10461?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvcm91dGluZy9zZWdtZW50cHJ1bmVyL1NpbmdsZVBhcnRpdGlvbkNvbHVtblNlZ21lbnRQcnVuZXIuamF2YQ==) | `61.90% <76.47%> (+0.68%)` | :arrow_up: |
   | [...oker/routing/timeboundary/TimeBoundaryManager.java](https://codecov.io/gh/apache/pinot/pull/10461?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvcm91dGluZy90aW1lYm91bmRhcnkvVGltZUJvdW5kYXJ5TWFuYWdlci5qYXZh) | `82.29% <76.92%> (+16.98%)` | :arrow_up: |
   | [...routing/segmentpartition/SegmentPartitionInfo.java](https://codecov.io/gh/apache/pinot/pull/10461?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvcm91dGluZy9zZWdtZW50cGFydGl0aW9uL1NlZ21lbnRQYXJ0aXRpb25JbmZvLmphdmE=) | `80.00% <80.00%> (ø)` | |
   | [...mentpruner/MultiPartitionColumnsSegmentPruner.java](https://codecov.io/gh/apache/pinot/pull/10461?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvcm91dGluZy9zZWdtZW50cHJ1bmVyL011bHRpUGFydGl0aW9uQ29sdW1uc1NlZ21lbnRQcnVuZXIuamF2YQ==) | `67.60% <86.36%> (+67.60%)` | :arrow_up: |
   | [...oker/routing/segmentpruner/EmptySegmentPruner.java](https://codecov.io/gh/apache/pinot/pull/10461?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvcm91dGluZy9zZWdtZW50cHJ1bmVyL0VtcHR5U2VnbWVudFBydW5lci5qYXZh) | `84.78% <87.50%> (+84.78%)` | :arrow_up: |
   | [...outing/segmentmetadata/SegmentZkMetadataCache.java](https://codecov.io/gh/apache/pinot/pull/10461?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvcm91dGluZy9zZWdtZW50bWV0YWRhdGEvU2VnbWVudFprTWV0YWRhdGFDYWNoZS5qYXZh) | `88.15% <88.15%> (ø)` | |
   | [...er/routing/segmentpruner/SegmentPrunerFactory.java](https://codecov.io/gh/apache/pinot/pull/10461?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvcm91dGluZy9zZWdtZW50cHJ1bmVyL1NlZ21lbnRQcnVuZXJGYWN0b3J5LmphdmE=) | `93.58% <100.00%> (+51.56%)` | :arrow_up: |
   | ... and [1 more](https://codecov.io/gh/apache/pinot/pull/10461?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ... and [921 files with indirect coverage changes](https://codecov.io/gh/apache/pinot/pull/10461/indirect-changes?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   :mega: We’re building smart automated test selection to slash your CI/CD build times. [Learn more](https://about.codecov.io/iterative-testing/?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   


-- 
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-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] walterddr commented on a diff in pull request #10461: add partition metadata manager

Posted by "walterddr (via GitHub)" <gi...@apache.org>.
walterddr commented on code in PR #10461:
URL: https://github.com/apache/pinot/pull/10461#discussion_r1152509846


##########
pinot-broker/src/main/java/org/apache/pinot/broker/routing/BrokerRoutingManager.java:
##########
@@ -478,20 +486,35 @@ public synchronized void buildRouting(String tableNameWithType) {
         Set<String> offlineTablePreSelectedOnlineSegments =
             offlineTableSegmentPreSelector.preSelect(offlineTableOnlineSegments);
         TimeBoundaryManager offlineTableTimeBoundaryManager =
-            new TimeBoundaryManager(offlineTableConfig, _propertyStore, _brokerMetrics);
-        offlineTableTimeBoundaryManager.init(offlineTableIdealState, offlineTableExternalView,
-            offlineTablePreSelectedOnlineSegments);
-        offlineTableRoutingEntry.setTimeBoundaryManager(offlineTableTimeBoundaryManager);
+            new TimeBoundaryManager(offlineTableConfig, schema, _brokerMetrics);
+        offlineTableRoutingEntry.initTimeBoundaryManager(offlineTableTimeBoundaryManager, offlineTableIdealState,
+            offlineTableExternalView, offlineTablePreSelectedOnlineSegments);

Review Comment:
   chatted offline with @Jackie-Jiang ^ we decided to leave timboundary out of the segment zk metadata fetcher. 
   1. timeboundary manager is init separately and can have different timing than the rest of the zk required utilities. 
   2. it is very possible that during server start up no zk metadata fetcher is needed (e.g. no pruner and others).
       - enabling this by default on all table is a waste
       - merging this with the rest will possible cause duplicate init and slow down server start up
   
   will comment on the javadoc to make sure this knowledge is documented after the PR is merged



-- 
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-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org