You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by GitBox <gi...@apache.org> on 2022/09/19 06:05:14 UTC

[GitHub] [pinot] navina opened a new pull request, #9424: refactor LLRealtimeSegmentDataManager

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

   There is no change in functionality in this PR. The changes are:
   * Created a wrapper class to encapsulate all the column descriptors used in `LLRealtimeSegmentDataManager` 
   * `LLRealtimeSegmentDataManager` should always use `createPartitionMetadataProvider` and not `createStreamMetadataProvider`


-- 
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] navina commented on a diff in pull request #9424: refactor LLRealtimeSegmentDataManager

Posted by GitBox <gi...@apache.org>.
navina commented on code in PR #9424:
URL: https://github.com/apache/pinot/pull/9424#discussion_r973934984


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/converter/ColumnDescriptionsContainer.java:
##########
@@ -0,0 +1,48 @@
+package org.apache.pinot.segment.local.realtime.converter;
+
+import java.util.List;
+
+
+public class ColumnDescriptionsContainer {
+  private final String _sortedColumn;
+  private final List<String> _invertedIndexColumns;
+  private final List<String> _textIndexColumns;
+  private final List<String> _fstIndexColumns;
+  private final List<String> _noDictionaryColumns;
+  private final List<String> _varLengthDictionaryColumns;
+
+  public ColumnDescriptionsContainer(String sortedColumn, List<String> invertedIndexColumns,
+      List<String> textIndexColumns, List<String> fstIndexColumns, List<String> noDictionaryColumns,
+      List<String> varLengthDictionaryColumns) {
+    _sortedColumn = sortedColumn;
+    _invertedIndexColumns = invertedIndexColumns;
+    _textIndexColumns = textIndexColumns;
+    _fstIndexColumns = fstIndexColumns;
+    _noDictionaryColumns = noDictionaryColumns;
+    _varLengthDictionaryColumns = varLengthDictionaryColumns;
+  }
+
+  public String getSortedColumn() {
+    return _sortedColumn;
+  }
+
+  public List<String> getInvertedIndexColumns() {
+    return _invertedIndexColumns;
+  }
+
+  public List<String> getTextIndexColumns() {
+    return _textIndexColumns;
+  }
+
+  public List<String> getFstIndexColumns() {
+    return _fstIndexColumns;
+  }
+
+  public List<String> getNoDictionaryColumns() {

Review Comment:
   Yeah. I wasn't sure why that is the case. It is being passed into `RealtimeSegmentConverter`. But even there, the member variable `_noDictionaryColumns` was unused. So, not sure why this exists. I didn't want to change any functionality in a refactoring PR. So, left it as is. 



-- 
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] navina commented on a diff in pull request #9424: refactor LLRealtimeSegmentDataManager

Posted by GitBox <gi...@apache.org>.
navina commented on code in PR #9424:
URL: https://github.com/apache/pinot/pull/9424#discussion_r974831127


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/converter/RealtimeSegmentConverter.java:
##########
@@ -49,34 +49,26 @@ public class RealtimeSegmentConverter {
   private final String _segmentName;
   private final String _sortedColumn;
   private final List<String> _invertedIndexColumns;
-  private final List<String> _textIndexColumns;
-  private final List<String> _fstIndexColumns;
-  private final List<String> _noDictionaryColumns;
-  private final List<String> _varLengthDictionaryColumns;
+  private final ColumnDescriptionsContainer _columnDescriptionsContainer;
   private final boolean _nullHandlingEnabled;
 
   public RealtimeSegmentConverter(MutableSegmentImpl realtimeSegment, SegmentZKPropsConfig segmentZKPropsConfig,
       String outputPath, Schema schema, String tableName, TableConfig tableConfig, String segmentName,
-      String sortedColumn, List<String> invertedIndexColumns, List<String> textIndexColumns,
-      List<String> fstIndexColumns, List<String> noDictionaryColumns, List<String> varLengthDictionaryColumns,
-      boolean nullHandlingEnabled) {
+      ColumnDescriptionsContainer cdc, boolean nullHandlingEnabled) {
     _realtimeSegmentImpl = realtimeSegment;
     _segmentZKPropsConfig = segmentZKPropsConfig;
     _outputPath = outputPath;
-    _invertedIndexColumns = new ArrayList<>(invertedIndexColumns);
-    if (sortedColumn != null) {
-      _invertedIndexColumns.remove(sortedColumn);
+    _columnDescriptionsContainer = cdc;
+    _invertedIndexColumns = new ArrayList<>(_columnDescriptionsContainer.getInvertedIndexColumns());
+    if (cdc.getSortedColumn() != null) {
+      _invertedIndexColumns.remove(cdc.getSortedColumn());
     }
     _dataSchema = getUpdatedSchema(schema);
-    _sortedColumn = sortedColumn;
+    _sortedColumn = cdc.getSortedColumn();
     _tableName = tableName;
     _tableConfig = tableConfig;
     _segmentName = segmentName;
-    _noDictionaryColumns = noDictionaryColumns;

Review Comment:
   I agree with the line of thought. But which variable are you referring to? 
   I had to keep `invertedIndexColumns` as it is being updated in this class and `ColumnIndicesForRealtimeTable` is meant to be used as an immutable container object.  Please clarify and I can make the changes. 



-- 
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] jackjlli commented on a diff in pull request #9424: refactor LLRealtimeSegmentDataManager

Posted by GitBox <gi...@apache.org>.
jackjlli commented on code in PR #9424:
URL: https://github.com/apache/pinot/pull/9424#discussion_r975609836


##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/LLRealtimeSegmentDataManager.java:
##########
@@ -1528,8 +1520,8 @@ private void makeStreamMetadataProvider(String reason) {
     if (_streamMetadataProvider != null) {
       closeStreamMetadataProvider();
     }
-    _segmentLogger.info("Creating new stream metadata provider, reason: {}", reason);
-    _streamMetadataProvider = _streamConsumerFactory.createStreamMetadataProvider(_clientId);
+    _segmentLogger.info("Creating new partition metadata provider, reason: {}", reason);
+    _streamMetadataProvider = _streamConsumerFactory.createPartitionMetadataProvider(_clientId, _partitionGroupId);

Review Comment:
   > I don't know why an interface called StreamMetadataProvider contains a method for both createStreamMetadataProvider (stream-level) and createPartitionMetadataProvider (partition-level).
   
   You mean `StreamConsumerFactory` right?
   
   Just taking a closer look, this class is called `HLRealtimeSegmentDataManager`, which is a high level data manager for realtime segment. So it should use the stream-level instead of the partition-level provider. @navina We should revert this change here.



-- 
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] jackjlli commented on a diff in pull request #9424: refactor LLRealtimeSegmentDataManager

Posted by GitBox <gi...@apache.org>.
jackjlli commented on code in PR #9424:
URL: https://github.com/apache/pinot/pull/9424#discussion_r975616199


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/converter/RealtimeSegmentConverter.java:
##########
@@ -49,34 +49,26 @@ public class RealtimeSegmentConverter {
   private final String _segmentName;
   private final String _sortedColumn;
   private final List<String> _invertedIndexColumns;
-  private final List<String> _textIndexColumns;
-  private final List<String> _fstIndexColumns;
-  private final List<String> _noDictionaryColumns;
-  private final List<String> _varLengthDictionaryColumns;
+  private final ColumnDescriptionsContainer _columnDescriptionsContainer;
   private final boolean _nullHandlingEnabled;
 
   public RealtimeSegmentConverter(MutableSegmentImpl realtimeSegment, SegmentZKPropsConfig segmentZKPropsConfig,
       String outputPath, Schema schema, String tableName, TableConfig tableConfig, String segmentName,
-      String sortedColumn, List<String> invertedIndexColumns, List<String> textIndexColumns,
-      List<String> fstIndexColumns, List<String> noDictionaryColumns, List<String> varLengthDictionaryColumns,
-      boolean nullHandlingEnabled) {
+      ColumnDescriptionsContainer cdc, boolean nullHandlingEnabled) {
     _realtimeSegmentImpl = realtimeSegment;
     _segmentZKPropsConfig = segmentZKPropsConfig;
     _outputPath = outputPath;
-    _invertedIndexColumns = new ArrayList<>(invertedIndexColumns);
-    if (sortedColumn != null) {
-      _invertedIndexColumns.remove(sortedColumn);
+    _columnDescriptionsContainer = cdc;
+    _invertedIndexColumns = new ArrayList<>(_columnDescriptionsContainer.getInvertedIndexColumns());
+    if (cdc.getSortedColumn() != null) {
+      _invertedIndexColumns.remove(cdc.getSortedColumn());
     }
     _dataSchema = getUpdatedSchema(schema);
-    _sortedColumn = sortedColumn;
+    _sortedColumn = cdc.getSortedColumn();
     _tableName = tableName;
     _tableConfig = tableConfig;
     _segmentName = segmentName;
-    _noDictionaryColumns = noDictionaryColumns;

Review Comment:
   I mean we should either only keep `_columnIndicesForRealtimeTable`, or only keep the other variables like `_invertedIndexColumns`, `_textIndexColumns`, etc. Keeping both in the same class makes it harder to maintain the code in the future. E.g. ppl can choose the index config either from the instance variable, or from the one in `_columnIndicesForRealtimeTable`. I prefer keeping them respectively as instance variables as it makes less confusion, and all the preprocessing procedures of those indices have been done in constructor before they are used in the class.



-- 
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] navina commented on a diff in pull request #9424: refactor LLRealtimeSegmentDataManager

Posted by GitBox <gi...@apache.org>.
navina commented on code in PR #9424:
URL: https://github.com/apache/pinot/pull/9424#discussion_r974829799


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/converter/ColumnDescriptionsContainer.java:
##########
@@ -0,0 +1,48 @@
+package org.apache.pinot.segment.local.realtime.converter;
+
+import java.util.List;
+
+
+public class ColumnDescriptionsContainer {

Review Comment:
   it is already under the `realtime` directory. So, it shouldn't require keeping "Realtime" in all class and variable names.J Just makes the code hard to read. 



-- 
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] navina commented on a diff in pull request #9424: refactor LLRealtimeSegmentDataManager

Posted by GitBox <gi...@apache.org>.
navina commented on code in PR #9424:
URL: https://github.com/apache/pinot/pull/9424#discussion_r976042844


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/converter/RealtimeSegmentConverter.java:
##########
@@ -49,34 +49,26 @@ public class RealtimeSegmentConverter {
   private final String _segmentName;
   private final String _sortedColumn;
   private final List<String> _invertedIndexColumns;
-  private final List<String> _textIndexColumns;
-  private final List<String> _fstIndexColumns;
-  private final List<String> _noDictionaryColumns;
-  private final List<String> _varLengthDictionaryColumns;
+  private final ColumnDescriptionsContainer _columnDescriptionsContainer;
   private final boolean _nullHandlingEnabled;
 
   public RealtimeSegmentConverter(MutableSegmentImpl realtimeSegment, SegmentZKPropsConfig segmentZKPropsConfig,
       String outputPath, Schema schema, String tableName, TableConfig tableConfig, String segmentName,
-      String sortedColumn, List<String> invertedIndexColumns, List<String> textIndexColumns,
-      List<String> fstIndexColumns, List<String> noDictionaryColumns, List<String> varLengthDictionaryColumns,
-      boolean nullHandlingEnabled) {
+      ColumnDescriptionsContainer cdc, boolean nullHandlingEnabled) {
     _realtimeSegmentImpl = realtimeSegment;
     _segmentZKPropsConfig = segmentZKPropsConfig;
     _outputPath = outputPath;
-    _invertedIndexColumns = new ArrayList<>(invertedIndexColumns);
-    if (sortedColumn != null) {
-      _invertedIndexColumns.remove(sortedColumn);
+    _columnDescriptionsContainer = cdc;
+    _invertedIndexColumns = new ArrayList<>(_columnDescriptionsContainer.getInvertedIndexColumns());
+    if (cdc.getSortedColumn() != null) {
+      _invertedIndexColumns.remove(cdc.getSortedColumn());
     }
     _dataSchema = getUpdatedSchema(schema);
-    _sortedColumn = sortedColumn;
+    _sortedColumn = cdc.getSortedColumn();
     _tableName = tableName;
     _tableConfig = tableConfig;
     _segmentName = segmentName;
-    _noDictionaryColumns = noDictionaryColumns;

Review Comment:
   Ah I see what you mean. I will make the changes to keep the `_columnIndicesForRealtimeTable` and remove the instance variables. 



-- 
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] navina commented on pull request #9424: refactor LLRealtimeSegmentDataManager

Posted by GitBox <gi...@apache.org>.
navina commented on PR #9424:
URL: https://github.com/apache/pinot/pull/9424#issuecomment-1250603338

   @Jackie-Jiang / @KKcorps / @npawar : please review! 


-- 
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] navina commented on a diff in pull request #9424: refactor LLRealtimeSegmentDataManager

Posted by GitBox <gi...@apache.org>.
navina commented on code in PR #9424:
URL: https://github.com/apache/pinot/pull/9424#discussion_r973933537


##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/LLRealtimeSegmentDataManager.java:
##########
@@ -288,7 +285,7 @@ public void deleteSegmentFile() {
   private final SegmentCommitterFactory _segmentCommitterFactory;
   private final ConsumptionRateLimiter _rateLimiter;
 
-  private volatile StreamPartitionMsgOffset _latestStreamOffsetAtStartupTime = null;
+  private final StreamPartitionMsgOffset _latestStreamOffsetAtStartupTime;

Review Comment:
   nope. It is set once during startup of the `LLRealtimeSegmentDataManager`. After that, it is only read from. 



-- 
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] jackjlli commented on a diff in pull request #9424: refactor LLRealtimeSegmentDataManager

Posted by GitBox <gi...@apache.org>.
jackjlli commented on code in PR #9424:
URL: https://github.com/apache/pinot/pull/9424#discussion_r980510492


##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/LLRealtimeSegmentDataManager.java:
##########
@@ -1435,10 +1428,13 @@ private void setConsumeEndTime(SegmentZKMetadata segmentZKMetadata, long now) {
   }
 
   public StreamPartitionMsgOffset fetchLatestStreamOffset(long maxWaitTimeMs) {
-    try (StreamMetadataProvider metadataProvider = _streamConsumerFactory.createPartitionMetadataProvider(_clientId,
-        _partitionGroupId)) {
-      return metadataProvider.fetchStreamPartitionOffset(OffsetCriteria.LARGEST_OFFSET_CRITERIA, maxWaitTimeMs);
-    } catch (Exception e) {
+    if (_partitionMetadataProvider == null) {
+      createPartitionMetadataProvider("Fetch latest stream offset");
+    }
+    try {
+      return _partitionMetadataProvider.fetchStreamPartitionOffset(OffsetCriteria.LARGEST_OFFSET_CRITERIA,

Review Comment:
   +1 on this. No need to instantiate another partitionLevelMetadataProvider here.



-- 
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 #9424: refactor LLRealtimeSegmentDataManager

Posted by GitBox <gi...@apache.org>.
codecov-commenter commented on PR #9424:
URL: https://github.com/apache/pinot/pull/9424#issuecomment-1251877692

   # [Codecov](https://codecov.io/gh/apache/pinot/pull/9424?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 [#9424](https://codecov.io/gh/apache/pinot/pull/9424?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (ee02a56) into [master](https://codecov.io/gh/apache/pinot/commit/2d6665b8e5fa0842ef67b3d9896c5e04ecad78e9?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (2d6665b) will **decrease** coverage by `2.66%`.
   > The diff coverage is `66.66%`.
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #9424      +/-   ##
   ============================================
   - Coverage     69.73%   67.07%   -2.67%     
   - Complexity     4787     4907     +120     
   ============================================
     Files          1890     1407     -483     
     Lines        100756    73339   -27417     
     Branches      15350    11726    -3624     
   ============================================
   - Hits          70266    49190   -21076     
   + Misses        25507    20589    -4918     
   + Partials       4983     3560    -1423     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration1 | `?` | |
   | integration2 | `?` | |
   | unittests1 | `67.07% <66.66%> (+0.06%)` | :arrow_up: |
   | unittests2 | `?` | |
   
   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/9424?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...manager/realtime/HLRealtimeSegmentDataManager.java](https://codecov.io/gh/apache/pinot/pull/9424/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9kYXRhL21hbmFnZXIvcmVhbHRpbWUvSExSZWFsdGltZVNlZ21lbnREYXRhTWFuYWdlci5qYXZh) | `0.00% <0.00%> (-82.80%)` | :arrow_down: |
   | [...manager/realtime/LLRealtimeSegmentDataManager.java](https://codecov.io/gh/apache/pinot/pull/9424/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9kYXRhL21hbmFnZXIvcmVhbHRpbWUvTExSZWFsdGltZVNlZ21lbnREYXRhTWFuYWdlci5qYXZh) | `52.71% <61.90%> (-17.90%)` | :arrow_down: |
   | [...l/realtime/converter/RealtimeSegmentConverter.java](https://codecov.io/gh/apache/pinot/pull/9424/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9yZWFsdGltZS9jb252ZXJ0ZXIvUmVhbHRpbWVTZWdtZW50Q29udmVydGVyLmphdmE=) | `79.62% <80.00%> (-1.08%)` | :arrow_down: |
   | [...ltime/converter/ColumnIndicesForRealtimeTable.java](https://codecov.io/gh/apache/pinot/pull/9424/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9yZWFsdGltZS9jb252ZXJ0ZXIvQ29sdW1uSW5kaWNlc0ZvclJlYWx0aW1lVGFibGUuamF2YQ==) | `92.85% <92.85%> (ø)` | |
   | [...va/org/apache/pinot/core/routing/RoutingTable.java](https://codecov.io/gh/apache/pinot/pull/9424/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9yb3V0aW5nL1JvdXRpbmdUYWJsZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...va/org/apache/pinot/common/config/NettyConfig.java](https://codecov.io/gh/apache/pinot/pull/9424/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vY29uZmlnL05ldHR5Q29uZmlnLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...a/org/apache/pinot/common/metrics/MinionMeter.java](https://codecov.io/gh/apache/pinot/pull/9424/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWV0cmljcy9NaW5pb25NZXRlci5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...g/apache/pinot/common/metrics/ControllerMeter.java](https://codecov.io/gh/apache/pinot/pull/9424/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWV0cmljcy9Db250cm9sbGVyTWV0ZXIuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../apache/pinot/common/metrics/BrokerQueryPhase.java](https://codecov.io/gh/apache/pinot/pull/9424/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWV0cmljcy9Ccm9rZXJRdWVyeVBoYXNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../apache/pinot/common/metrics/MinionQueryPhase.java](https://codecov.io/gh/apache/pinot/pull/9424/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWV0cmljcy9NaW5pb25RdWVyeVBoYXNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | ... and [739 more](https://codecov.io/gh/apache/pinot/pull/9424/diff?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] KKcorps commented on a diff in pull request #9424: refactor LLRealtimeSegmentDataManager

Posted by GitBox <gi...@apache.org>.
KKcorps commented on code in PR #9424:
URL: https://github.com/apache/pinot/pull/9424#discussion_r973900800


##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/LLRealtimeSegmentDataManager.java:
##########
@@ -288,7 +285,7 @@ public void deleteSegmentFile() {
   private final SegmentCommitterFactory _segmentCommitterFactory;
   private final ConsumptionRateLimiter _rateLimiter;
 
-  private volatile StreamPartitionMsgOffset _latestStreamOffsetAtStartupTime = null;
+  private final StreamPartitionMsgOffset _latestStreamOffsetAtStartupTime;

Review Comment:
   Is this variable not being modified anywhere?



##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/converter/ColumnDescriptionsContainer.java:
##########
@@ -0,0 +1,48 @@
+package org.apache.pinot.segment.local.realtime.converter;
+
+import java.util.List;
+
+
+public class ColumnDescriptionsContainer {
+  private final String _sortedColumn;
+  private final List<String> _invertedIndexColumns;
+  private final List<String> _textIndexColumns;
+  private final List<String> _fstIndexColumns;
+  private final List<String> _noDictionaryColumns;
+  private final List<String> _varLengthDictionaryColumns;
+
+  public ColumnDescriptionsContainer(String sortedColumn, List<String> invertedIndexColumns,
+      List<String> textIndexColumns, List<String> fstIndexColumns, List<String> noDictionaryColumns,
+      List<String> varLengthDictionaryColumns) {
+    _sortedColumn = sortedColumn;
+    _invertedIndexColumns = invertedIndexColumns;
+    _textIndexColumns = textIndexColumns;
+    _fstIndexColumns = fstIndexColumns;
+    _noDictionaryColumns = noDictionaryColumns;
+    _varLengthDictionaryColumns = varLengthDictionaryColumns;
+  }
+
+  public String getSortedColumn() {
+    return _sortedColumn;
+  }
+
+  public List<String> getInvertedIndexColumns() {
+    return _invertedIndexColumns;
+  }
+
+  public List<String> getTextIndexColumns() {
+    return _textIndexColumns;
+  }
+
+  public List<String> getFstIndexColumns() {
+    return _fstIndexColumns;
+  }
+
+  public List<String> getNoDictionaryColumns() {

Review Comment:
   Seems like this method is not being used anywhere but it should ideally be used.



##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/LLRealtimeSegmentDataManager.java:
##########
@@ -1529,7 +1521,7 @@ private void makeStreamMetadataProvider(String reason) {
       closeStreamMetadataProvider();
     }
     _segmentLogger.info("Creating new stream metadata provider, reason: {}", reason);

Review Comment:
   ```suggestion
       _segmentLogger.info("Creating new partition metadata provider, reason: {}", reason);
   ```



-- 
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] npawar commented on a diff in pull request #9424: refactor LLRealtimeSegmentDataManager

Posted by GitBox <gi...@apache.org>.
npawar commented on code in PR #9424:
URL: https://github.com/apache/pinot/pull/9424#discussion_r976845348


##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/LLRealtimeSegmentDataManager.java:
##########
@@ -1524,12 +1519,10 @@ private void recreateStreamConsumer(String reason) {
   /**
    * Creates a new stream metadata provider
    */
-  private void makeStreamMetadataProvider(String reason) {
-    if (_streamMetadataProvider != null) {
-      closeStreamMetadataProvider();
-    }
-    _segmentLogger.info("Creating new stream metadata provider, reason: {}", reason);
-    _streamMetadataProvider = _streamConsumerFactory.createStreamMetadataProvider(_clientId);
+  private void createPartitionMetadataProvider(String reason) {
+    closeStreamMetadataProvider();

Review Comment:
   nit: s/_streamMetadataProvider/_partitionMetadataProvider and s/closeStreamMetadataProvider/closePartitionMetadataProvider ?



-- 
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] jackjlli merged pull request #9424: refactor LLRealtimeSegmentDataManager

Posted by GitBox <gi...@apache.org>.
jackjlli merged PR #9424:
URL: https://github.com/apache/pinot/pull/9424


-- 
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] jackjlli commented on a diff in pull request #9424: refactor LLRealtimeSegmentDataManager

Posted by GitBox <gi...@apache.org>.
jackjlli commented on code in PR #9424:
URL: https://github.com/apache/pinot/pull/9424#discussion_r975609836


##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/LLRealtimeSegmentDataManager.java:
##########
@@ -1528,8 +1520,8 @@ private void makeStreamMetadataProvider(String reason) {
     if (_streamMetadataProvider != null) {
       closeStreamMetadataProvider();
     }
-    _segmentLogger.info("Creating new stream metadata provider, reason: {}", reason);
-    _streamMetadataProvider = _streamConsumerFactory.createStreamMetadataProvider(_clientId);
+    _segmentLogger.info("Creating new partition metadata provider, reason: {}", reason);
+    _streamMetadataProvider = _streamConsumerFactory.createPartitionMetadataProvider(_clientId, _partitionGroupId);

Review Comment:
   > I don't know why an interface called StreamMetadataProvider contains a method for both createStreamMetadataProvider (stream-level) and createPartitionMetadataProvider (partition-level).
   
   I think it's because the high-level segment data manager was implemented before the low-level segment data manager. But since the caller method is a private class, we can make it more intuitive to understand by just renaming it.
   



-- 
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] jackjlli commented on a diff in pull request #9424: refactor LLRealtimeSegmentDataManager

Posted by GitBox <gi...@apache.org>.
jackjlli commented on code in PR #9424:
URL: https://github.com/apache/pinot/pull/9424#discussion_r974672289


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/converter/RealtimeSegmentConverter.java:
##########
@@ -49,34 +49,26 @@ public class RealtimeSegmentConverter {
   private final String _segmentName;
   private final String _sortedColumn;
   private final List<String> _invertedIndexColumns;
-  private final List<String> _textIndexColumns;
-  private final List<String> _fstIndexColumns;
-  private final List<String> _noDictionaryColumns;
-  private final List<String> _varLengthDictionaryColumns;
+  private final ColumnDescriptionsContainer _columnDescriptionsContainer;
   private final boolean _nullHandlingEnabled;
 
   public RealtimeSegmentConverter(MutableSegmentImpl realtimeSegment, SegmentZKPropsConfig segmentZKPropsConfig,
       String outputPath, Schema schema, String tableName, TableConfig tableConfig, String segmentName,
-      String sortedColumn, List<String> invertedIndexColumns, List<String> textIndexColumns,
-      List<String> fstIndexColumns, List<String> noDictionaryColumns, List<String> varLengthDictionaryColumns,
-      boolean nullHandlingEnabled) {
+      ColumnDescriptionsContainer cdc, boolean nullHandlingEnabled) {
     _realtimeSegmentImpl = realtimeSegment;
     _segmentZKPropsConfig = segmentZKPropsConfig;
     _outputPath = outputPath;
-    _invertedIndexColumns = new ArrayList<>(invertedIndexColumns);
-    if (sortedColumn != null) {
-      _invertedIndexColumns.remove(sortedColumn);
+    _columnDescriptionsContainer = cdc;
+    _invertedIndexColumns = new ArrayList<>(_columnDescriptionsContainer.getInvertedIndexColumns());
+    if (cdc.getSortedColumn() != null) {
+      _invertedIndexColumns.remove(cdc.getSortedColumn());
     }
     _dataSchema = getUpdatedSchema(schema);
-    _sortedColumn = sortedColumn;
+    _sortedColumn = cdc.getSortedColumn();
     _tableName = tableName;
     _tableConfig = tableConfig;
     _segmentName = segmentName;
-    _noDictionaryColumns = noDictionaryColumns;

Review Comment:
   I think we should consider either adding an instance level variable just for columnDescriptionsContainer object, or assigning all the necessary indices into instance variables, instead of keeping both here. Otherwise, it's a bit harder to maintain in the future.



-- 
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] navina commented on a diff in pull request #9424: refactor LLRealtimeSegmentDataManager

Posted by GitBox <gi...@apache.org>.
navina commented on code in PR #9424:
URL: https://github.com/apache/pinot/pull/9424#discussion_r976070414


##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/LLRealtimeSegmentDataManager.java:
##########
@@ -1528,8 +1520,8 @@ private void makeStreamMetadataProvider(String reason) {
     if (_streamMetadataProvider != null) {
       closeStreamMetadataProvider();
     }
-    _segmentLogger.info("Creating new stream metadata provider, reason: {}", reason);
-    _streamMetadataProvider = _streamConsumerFactory.createStreamMetadataProvider(_clientId);
+    _segmentLogger.info("Creating new partition metadata provider, reason: {}", reason);
+    _streamMetadataProvider = _streamConsumerFactory.createPartitionMetadataProvider(_clientId, _partitionGroupId);

Review Comment:
   Ok. I renamed the method to `createPartitionMetadataProvider` 



-- 
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] jackjlli commented on a diff in pull request #9424: refactor LLRealtimeSegmentDataManager

Posted by GitBox <gi...@apache.org>.
jackjlli commented on code in PR #9424:
URL: https://github.com/apache/pinot/pull/9424#discussion_r976816855


##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/LLRealtimeSegmentDataManager.java:
##########
@@ -1528,8 +1520,8 @@ private void makeStreamMetadataProvider(String reason) {
     if (_streamMetadataProvider != null) {
       closeStreamMetadataProvider();
     }
-    _segmentLogger.info("Creating new stream metadata provider, reason: {}", reason);
-    _streamMetadataProvider = _streamConsumerFactory.createStreamMetadataProvider(_clientId);
+    _segmentLogger.info("Creating new partition metadata provider, reason: {}", reason);
+    _streamMetadataProvider = _streamConsumerFactory.createPartitionMetadataProvider(_clientId, _partitionGroupId);

Review Comment:
   Sorry for the late comment here. Somehow I forgot to submit the review yesterday. 
   
   By taking a closer look, the purpose of `_streamMetadataProvider` is to fetch the latest value of partitionCount from the stream provider, which shouldn't associate with any particular partitionId here. We should stick to using the stream-level stream metadata provider here.



-- 
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] navina commented on a diff in pull request #9424: refactor LLRealtimeSegmentDataManager

Posted by GitBox <gi...@apache.org>.
navina commented on code in PR #9424:
URL: https://github.com/apache/pinot/pull/9424#discussion_r977496047


##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/LLRealtimeSegmentDataManager.java:
##########
@@ -1524,12 +1519,10 @@ private void recreateStreamConsumer(String reason) {
   /**
    * Creates a new stream metadata provider
    */
-  private void makeStreamMetadataProvider(String reason) {
-    if (_streamMetadataProvider != null) {
-      closeStreamMetadataProvider();
-    }
-    _segmentLogger.info("Creating new stream metadata provider, reason: {}", reason);
-    _streamMetadataProvider = _streamConsumerFactory.createStreamMetadataProvider(_clientId);
+  private void createPartitionMetadataProvider(String reason) {
+    closeStreamMetadataProvider();

Review Comment:
   done



-- 
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] jackjlli commented on a diff in pull request #9424: refactor LLRealtimeSegmentDataManager

Posted by GitBox <gi...@apache.org>.
jackjlli commented on code in PR #9424:
URL: https://github.com/apache/pinot/pull/9424#discussion_r974667348


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/converter/ColumnDescriptionsContainer.java:
##########
@@ -0,0 +1,48 @@
+package org.apache.pinot.segment.local.realtime.converter;

Review Comment:
   Include the header into this class.



##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/LLRealtimeSegmentDataManager.java:
##########
@@ -1528,8 +1520,8 @@ private void makeStreamMetadataProvider(String reason) {
     if (_streamMetadataProvider != null) {
       closeStreamMetadataProvider();
     }
-    _segmentLogger.info("Creating new stream metadata provider, reason: {}", reason);
-    _streamMetadataProvider = _streamConsumerFactory.createStreamMetadataProvider(_clientId);
+    _segmentLogger.info("Creating new partition metadata provider, reason: {}", reason);
+    _streamMetadataProvider = _streamConsumerFactory.createPartitionMetadataProvider(_clientId, _partitionGroupId);

Review Comment:
   Should we consider renaming the caller method name?



##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/converter/RealtimeSegmentConverter.java:
##########
@@ -49,34 +49,26 @@ public class RealtimeSegmentConverter {
   private final String _segmentName;
   private final String _sortedColumn;
   private final List<String> _invertedIndexColumns;
-  private final List<String> _textIndexColumns;
-  private final List<String> _fstIndexColumns;
-  private final List<String> _noDictionaryColumns;
-  private final List<String> _varLengthDictionaryColumns;
+  private final ColumnDescriptionsContainer _columnDescriptionsContainer;
   private final boolean _nullHandlingEnabled;
 
   public RealtimeSegmentConverter(MutableSegmentImpl realtimeSegment, SegmentZKPropsConfig segmentZKPropsConfig,
       String outputPath, Schema schema, String tableName, TableConfig tableConfig, String segmentName,
-      String sortedColumn, List<String> invertedIndexColumns, List<String> textIndexColumns,
-      List<String> fstIndexColumns, List<String> noDictionaryColumns, List<String> varLengthDictionaryColumns,
-      boolean nullHandlingEnabled) {
+      ColumnDescriptionsContainer cdc, boolean nullHandlingEnabled) {
     _realtimeSegmentImpl = realtimeSegment;
     _segmentZKPropsConfig = segmentZKPropsConfig;
     _outputPath = outputPath;
-    _invertedIndexColumns = new ArrayList<>(invertedIndexColumns);
-    if (sortedColumn != null) {
-      _invertedIndexColumns.remove(sortedColumn);
+    _columnDescriptionsContainer = cdc;
+    _invertedIndexColumns = new ArrayList<>(_columnDescriptionsContainer.getInvertedIndexColumns());
+    if (cdc.getSortedColumn() != null) {
+      _invertedIndexColumns.remove(cdc.getSortedColumn());
     }
     _dataSchema = getUpdatedSchema(schema);
-    _sortedColumn = sortedColumn;
+    _sortedColumn = cdc.getSortedColumn();
     _tableName = tableName;
     _tableConfig = tableConfig;
     _segmentName = segmentName;
-    _noDictionaryColumns = noDictionaryColumns;

Review Comment:
   I think we should either adding an instance level variable just for columnDescriptionsContainer object, or assigning all the necessary indices into instance variables, instead of keeping both here. Otherwise, it's a bit harder to maintain in the future.



##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/HLRealtimeSegmentDataManager.java:
##########
@@ -284,11 +285,12 @@ public void run() {
           _segmentLogger.info("Indexed {} raw events", _realtimeSegment.getNumDocsIndexed());
           File tempSegmentFolder = new File(_resourceTmpDir, "tmp-" + System.currentTimeMillis());
           // lets convert the segment now
+          ColumnDescriptionsContainer cdc = new ColumnDescriptionsContainer(_sortedColumn, _invertedIndexColumns,

Review Comment:
   nit: it'd be good to move this before the comments in Line 287? Also, use `columnDescriptionsContainer` as the variable name instead?



##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/converter/ColumnDescriptionsContainer.java:
##########
@@ -0,0 +1,48 @@
+package org.apache.pinot.segment.local.realtime.converter;
+
+import java.util.List;
+
+
+public class ColumnDescriptionsContainer {

Review Comment:
   nit: maybe coming up with a better name like `ColumnIndicesForRealtimeTable` and the like, as it's just for realtime table?



##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/converter/ColumnDescriptionsContainer.java:
##########
@@ -0,0 +1,48 @@
+package org.apache.pinot.segment.local.realtime.converter;
+
+import java.util.List;
+
+
+public class ColumnDescriptionsContainer {

Review Comment:
   Add javadoc into this class.



-- 
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] navina commented on a diff in pull request #9424: refactor LLRealtimeSegmentDataManager

Posted by GitBox <gi...@apache.org>.
navina commented on code in PR #9424:
URL: https://github.com/apache/pinot/pull/9424#discussion_r974828374


##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/LLRealtimeSegmentDataManager.java:
##########
@@ -1528,8 +1520,8 @@ private void makeStreamMetadataProvider(String reason) {
     if (_streamMetadataProvider != null) {
       closeStreamMetadataProvider();
     }
-    _segmentLogger.info("Creating new stream metadata provider, reason: {}", reason);
-    _streamMetadataProvider = _streamConsumerFactory.createStreamMetadataProvider(_clientId);
+    _segmentLogger.info("Creating new partition metadata provider, reason: {}", reason);
+    _streamMetadataProvider = _streamConsumerFactory.createPartitionMetadataProvider(_clientId, _partitionGroupId);

Review Comment:
   I don't know why an interface called `StreamMetadataProvider` contains a method for both `createStreamMetadataProvider` (stream-level) and `createPartitionMetadataProvider` (partition-level). I am sure there some history to do this. 
   what is your proposal here? 
   



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