mcvsubbu commented on a change in pull request #6667:
URL: https://github.com/apache/incubator-pinot/pull/6667#discussion_r593325379



##########
File path: 
pinot-common/src/main/java/org/apache/pinot/common/metadata/segment/LLCRealtimeSegmentZKMetadata.java
##########
@@ -87,11 +87,6 @@ public void setDownloadUrl(String downloadUrl) {
   public ZNRecord toZNRecord() {
     ZNRecord znRecord = super.toZNRecord();
     znRecord.setSimpleField(START_OFFSET, _startOffset);
-    if (_endOffset == null) {
-      // TODO Issue 5359 Keep this until all components have upgraded to a 
version that can handle _offset being null

Review comment:
       Perhaps we should make a note that before this upgrade, installations 
should upgrade to at least the release that has 
https://github.com/apache/incubator-pinot/issues/5359 (Identify the actual 
release in the PR comments). An installation that has older versions than pr 
5359 may break during upgrade.

##########
File path: 
pinot-common/src/main/java/org/apache/pinot/common/protocols/SegmentCompletionProtocol.java
##########
@@ -180,6 +184,16 @@ private Request(Params params, String msgType) {
     }
 
     public String getUrl(String hostPort, String protocol) {
+      String streamPartitionMsgOffset;
+      try {
+        streamPartitionMsgOffset = _params.getStreamPartitionMsgOffset() == 
null ? null :
+            URLEncoder.encode(_params.getStreamPartitionMsgOffset(), 
StandardCharsets.UTF_8.toString());

Review comment:
       If Stream partition offset needs to be url encoded then that logic has 
to be specific to the stream partition offset of the stream implementation. 
Please move it there. As far as the protocol is concerned, it should be a 
string.
   
   Protocol getters should not be throwing exceptions

##########
File path: 
pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotTableIdealStateBuilder.java
##########
@@ -115,14 +117,22 @@ public static void 
buildLowLevelRealtimeIdealStateFor(PinotLLCRealtimeSegmentMan
     pinotLLCRealtimeSegmentManager.setUpNewTable(realtimeTableConfig, 
idealState);
   }
 
-  public static int getPartitionCount(StreamConfig streamConfig) {
-    PartitionCountFetcher partitionCountFetcher = new 
PartitionCountFetcher(streamConfig);
+  /**
+   * Fetches the list of {@link PartitionGroupInfo} for the stream, with the 
help of the current partitionGroups metadata
+   * This call will only skip partitions which have reached end of life and 
all messages from that partition have been consumed.
+   * The current partition group metadata is used to determine the offsets 
that have been consumed for a partition.
+   * The current partition group metadata is also used to know about existing 
partition groupings which should not be disturbed
+   */
+  public static List<PartitionGroupInfo> 
getPartitionGroupInfoList(StreamConfig streamConfig,
+      List<PartitionGroupMetadata> currentPartitionGroupMetadataList) {

Review comment:
       `PartitionGroupMetadata` seems to be one per segment. Can you clarify 
why we need the existing segment metadata in order to find the info about 
existing partition groups in the stream? 
   Also, does `currentPartitionGroupMetadataList` contain the metadata for all 
segments? Only latest segments? Only consuming segments? Latest online 
segments?  Some documentation will be useful.

##########
File path: 
pinot-spi/src/main/java/org/apache/pinot/spi/stream/PartitionGroupMetadata.java
##########
@@ -0,0 +1,76 @@
+/**
+ * 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.spi.stream;
+
+/**
+ * A wrapper around all the information of a current Partition Group 

Review comment:
       Not really. This seems to be an extension of segment metadata. AFAICS 
there is a 1-1 correspondence between a segment and partition group metadata. 
What is not clear is whether it also exists for CONSUMING segments

##########
File path: 
pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotTableIdealStateBuilder.java
##########
@@ -115,14 +117,22 @@ public static void 
buildLowLevelRealtimeIdealStateFor(PinotLLCRealtimeSegmentMan
     pinotLLCRealtimeSegmentManager.setUpNewTable(realtimeTableConfig, 
idealState);
   }
 
-  public static int getPartitionCount(StreamConfig streamConfig) {
-    PartitionCountFetcher partitionCountFetcher = new 
PartitionCountFetcher(streamConfig);
+  /**
+   * Fetches the list of {@link PartitionGroupInfo} for the stream, with the 
help of the current partitionGroups metadata
+   * This call will only skip partitions which have reached end of life and 
all messages from that partition have been consumed.

Review comment:
       ```suggestion
      * This call will only skip a partition group when they have reached end 
of life and all messages from that partition group have been consumed.
   ```

##########
File path: 
pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotTableIdealStateBuilder.java
##########
@@ -115,14 +117,22 @@ public static void 
buildLowLevelRealtimeIdealStateFor(PinotLLCRealtimeSegmentMan
     pinotLLCRealtimeSegmentManager.setUpNewTable(realtimeTableConfig, 
idealState);
   }
 
-  public static int getPartitionCount(StreamConfig streamConfig) {
-    PartitionCountFetcher partitionCountFetcher = new 
PartitionCountFetcher(streamConfig);
+  /**
+   * Fetches the list of {@link PartitionGroupInfo} for the stream, with the 
help of the current partitionGroups metadata
+   * This call will only skip partitions which have reached end of life and 
all messages from that partition have been consumed.
+   * The current partition group metadata is used to determine the offsets 
that have been consumed for a partition.
+   * The current partition group metadata is also used to know about existing 
partition groupings which should not be disturbed
+   */
+  public static List<PartitionGroupInfo> 
getPartitionGroupInfoList(StreamConfig streamConfig,
+      List<PartitionGroupMetadata> currentPartitionGroupMetadataList) {
+    PartitionGroupInfoFetcher partitionGroupInfoFetcher =
+        new PartitionGroupInfoFetcher(streamConfig, 
currentPartitionGroupMetadataList);
     try {
-      RetryPolicies.noDelayRetryPolicy(3).attempt(partitionCountFetcher);
-      return partitionCountFetcher.getPartitionCount();
+      RetryPolicies.noDelayRetryPolicy(3).attempt(partitionGroupInfoFetcher);
+      return partitionGroupInfoFetcher.getPartitionGroupInfoList();
     } catch (Exception e) {
-      Exception fetcherException = partitionCountFetcher.getException();
-      LOGGER.error("Could not get partition count for {}", 
streamConfig.getTopicName(), fetcherException);
+      Exception fetcherException = partitionGroupInfoFetcher.getException();
+      LOGGER.error("Could not get partition group info for {}", 
streamConfig.getTopicName(), fetcherException);

Review comment:
       I know it was not there before, but can you please add the table name to 
this log? Thank you.

##########
File path: 
pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotTableIdealStateBuilder.java
##########
@@ -115,14 +117,22 @@ public static void 
buildLowLevelRealtimeIdealStateFor(PinotLLCRealtimeSegmentMan
     pinotLLCRealtimeSegmentManager.setUpNewTable(realtimeTableConfig, 
idealState);
   }
 
-  public static int getPartitionCount(StreamConfig streamConfig) {
-    PartitionCountFetcher partitionCountFetcher = new 
PartitionCountFetcher(streamConfig);
+  /**
+   * Fetches the list of {@link PartitionGroupInfo} for the stream, with the 
help of the current partitionGroups metadata
+   * This call will only skip partitions which have reached end of life and 
all messages from that partition have been consumed.
+   * The current partition group metadata is used to determine the offsets 
that have been consumed for a partition.
+   * The current partition group metadata is also used to know about existing 
partition groupings which should not be disturbed

Review comment:
       Can you clarify what is meant by "groupings" here? By any chance do we 
plan to look under the hood of a partition group to determine the name of the 
partitions in it? Otherwise,  let us just word it as "groups"




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

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

Reply via email to