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

[GitHub] [pinot] jtao15 opened a new pull request, #10754: Keep last completed segments for retention

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

   Do not delete last completed segments in retention manager such that there's no issue to create new consuming segments (e.g. the table is paused).
   


-- 
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] sajjad-moradi commented on a diff in pull request #10754: Keep last completed segments for retention

Posted by "sajjad-moradi (via GitHub)" <gi...@apache.org>.
sajjad-moradi commented on code in PR #10754:
URL: https://github.com/apache/pinot/pull/10754#discussion_r1195998765


##########
pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/retention/RetentionManagerTest.java:
##########
@@ -233,6 +233,39 @@ public void testRealtimeLLCCleanup()
     verify(pinotHelixResourceManager, times(1)).deleteSegments(anyString(), anyList());
   }
 
+  // This test makes sure that we do not clean up last llc completed segments
+  @Test
+  public void testRealtimeLastLLCCleanup()
+      throws Exception {
+    final long now = System.currentTimeMillis();
+    final int replicaCount = 1;
+
+    TableConfig tableConfig = createRealtimeTableConfig1(replicaCount);
+    List<String> removedSegments = new ArrayList<>();
+    LeadControllerManager leadControllerManager = mock(LeadControllerManager.class);
+    when(leadControllerManager.isLeaderForTable(anyString())).thenReturn(true);
+    PinotHelixResourceManager pinotHelixResourceManager =
+        setupSegmentMetadataForPausedTable(tableConfig, now, removedSegments);
+    setupPinotHelixResourceManager(tableConfig, removedSegments, pinotHelixResourceManager, leadControllerManager);
+
+    ControllerConf conf = new ControllerConf();
+    ControllerMetrics controllerMetrics = new ControllerMetrics(PinotMetricUtils.getPinotMetricsRegistry());
+    conf.setRetentionControllerFrequencyInSeconds(0);
+    conf.setDeletedSegmentsRetentionInDays(0);
+    RetentionManager retentionManager =
+        new RetentionManager(pinotHelixResourceManager, leadControllerManager, conf, controllerMetrics);
+    retentionManager.start();
+    retentionManager.run();
+
+    SegmentDeletionManager deletionManager = pinotHelixResourceManager.getSegmentDeletionManager();
+
+    // Verify that the removeAgedDeletedSegments() method in deletion manager is actually called.
+    verify(deletionManager, times(1)).removeAgedDeletedSegments(leadControllerManager);
+
+    // Verify that the deleteSegments method is actually called.
+    verify(pinotHelixResourceManager, times(1)).deleteSegments(anyString(), anyList());

Review Comment:
   ```suggestion
       String tableNameWithType = tableConfig.getTableName();
       String deletedSegmentName = new LLCSegmentName("testTable", 5, 0, now).getSegmentName();
       verify(pinotHelixResourceManager, times(1)).deleteSegments(tableNameWithType, ImmutableList.of(deletedSegmentName));
   ```



##########
pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManager.java:
##########
@@ -872,6 +874,24 @@ public List<SegmentZKMetadata> getSegmentsZKMetadata(String tableNameWithType) {
     return ZKMetadataProvider.getSegmentsZKMetadata(_propertyStore, tableNameWithType);
   }
 
+  public List<String> getLastLLCCompletedSegments(String tableNameWithType) {
+    Map<Integer, String> partitionIdToLastLLCCompletedSegmentMap = new HashMap<>();
+    List<SegmentZKMetadata> realtimeSegments = getSegmentsZKMetadata(tableNameWithType);
+    for (SegmentZKMetadata segMetadata : realtimeSegments) {
+      if (SegmentName.isLowLevelConsumerSegmentName(segMetadata.getSegmentName())
+          && segMetadata.getStatus() == CommonConstants.Segment.Realtime.Status.DONE) {
+        LLCSegmentName llcName = LLCSegmentName.of(segMetadata.getSegmentName());
+        int partitionGroupId = llcName.getPartitionGroupId();
+        int sequenceNumber = llcName.getSequenceNumber();
+        String curSegName = partitionIdToLastLLCCompletedSegmentMap.get(partitionGroupId);

Review Comment:
   Maybe `s/curSegName/lastCompletedSegName/`



##########
pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManager.java:
##########
@@ -872,6 +874,24 @@ public List<SegmentZKMetadata> getSegmentsZKMetadata(String tableNameWithType) {
     return ZKMetadataProvider.getSegmentsZKMetadata(_propertyStore, tableNameWithType);
   }
 
+  public List<String> getLastLLCCompletedSegments(String tableNameWithType) {
+    Map<Integer, String> partitionIdToLastLLCCompletedSegmentMap = new HashMap<>();
+    List<SegmentZKMetadata> realtimeSegments = getSegmentsZKMetadata(tableNameWithType);
+    for (SegmentZKMetadata segMetadata : realtimeSegments) {

Review Comment:
   ```suggestion
       for (SegmentZKMetadata segMetadata : getSegmentsZKMetadata(tableNameWithType)) {
   ```



-- 
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] jtao15 closed pull request #10754: Keep last completed segments for retention

Posted by "jtao15 (via GitHub)" <gi...@apache.org>.
jtao15 closed pull request #10754: Keep last completed segments for retention
URL: https://github.com/apache/pinot/pull/10754


-- 
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] jtao15 commented on pull request #10754: Keep last completed segments for retention

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

   > Could you elaborate a bit on the issue we faced in the PR description section?
   
   Updated the description to make it clearer. Please check and see if it makes sense to you.


-- 
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] jtao15 commented on a diff in pull request #10754: Keep last completed segments for retention

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


##########
pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManager.java:
##########
@@ -872,6 +874,24 @@ public List<SegmentZKMetadata> getSegmentsZKMetadata(String tableNameWithType) {
     return ZKMetadataProvider.getSegmentsZKMetadata(_propertyStore, tableNameWithType);
   }
 
+  public List<String> getLastLLCCompletedSegments(String tableNameWithType) {
+    Map<Integer, String> partitionIdToLastLLCCompletedSegmentMap = new HashMap<>();
+    for (SegmentZKMetadata segMetadata : getSegmentsZKMetadata(tableNameWithType)) {
+      if (SegmentName.isLowLevelConsumerSegmentName(segMetadata.getSegmentName())
+          && segMetadata.getStatus() == CommonConstants.Segment.Realtime.Status.DONE) {
+        LLCSegmentName llcName = LLCSegmentName.of(segMetadata.getSegmentName());
+        int partitionGroupId = llcName.getPartitionGroupId();
+        int sequenceNumber = llcName.getSequenceNumber();
+        String lastCompletedSegName = partitionIdToLastLLCCompletedSegmentMap.get(partitionGroupId);
+        if (lastCompletedSegName == null
+            || LLCSegmentName.of(lastCompletedSegName).getSequenceNumber() < sequenceNumber) {
+          partitionIdToLastLLCCompletedSegmentMap.put(partitionGroupId, segMetadata.getSegmentName());
+        }
+      }
+    }
+    return new ArrayList<>(partitionIdToLastLLCCompletedSegmentMap.values());

Review Comment:
   Removed the ArrayList and updated the return type as well.



-- 
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] jtao15 commented on a diff in pull request #10754: Keep last completed segments for retention

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


##########
pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManager.java:
##########
@@ -872,6 +874,24 @@ public List<SegmentZKMetadata> getSegmentsZKMetadata(String tableNameWithType) {
     return ZKMetadataProvider.getSegmentsZKMetadata(_propertyStore, tableNameWithType);
   }
 
+  public List<String> getLastLLCCompletedSegments(String tableNameWithType) {
+    Map<Integer, String> partitionIdToLastLLCCompletedSegmentMap = new HashMap<>();
+    for (SegmentZKMetadata segMetadata : getSegmentsZKMetadata(tableNameWithType)) {

Review Comment:
   `ZKMetadataProvider.getLLCRealtimeSegments()` only return the segment names, I want to get the metadata and check the status 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] jtao15 closed pull request #10754: Keep last completed segments for retention

Posted by "jtao15 (via GitHub)" <gi...@apache.org>.
jtao15 closed pull request #10754: Keep last completed segments for retention
URL: https://github.com/apache/pinot/pull/10754


-- 
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] jtao15 commented on a diff in pull request #10754: Keep last completed segments for retention

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


##########
pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/retention/RetentionManagerTest.java:
##########
@@ -233,6 +233,39 @@ public void testRealtimeLLCCleanup()
     verify(pinotHelixResourceManager, times(1)).deleteSegments(anyString(), anyList());
   }
 
+  // This test makes sure that we do not clean up last llc completed segments
+  @Test
+  public void testRealtimeLastLLCCleanup()
+      throws Exception {
+    final long now = System.currentTimeMillis();
+    final int replicaCount = 1;
+
+    TableConfig tableConfig = createRealtimeTableConfig1(replicaCount);
+    List<String> removedSegments = new ArrayList<>();
+    LeadControllerManager leadControllerManager = mock(LeadControllerManager.class);
+    when(leadControllerManager.isLeaderForTable(anyString())).thenReturn(true);
+    PinotHelixResourceManager pinotHelixResourceManager =
+        setupSegmentMetadataForPausedTable(tableConfig, now, removedSegments);
+    setupPinotHelixResourceManager(tableConfig, removedSegments, pinotHelixResourceManager, leadControllerManager);
+
+    ControllerConf conf = new ControllerConf();
+    ControllerMetrics controllerMetrics = new ControllerMetrics(PinotMetricUtils.getPinotMetricsRegistry());
+    conf.setRetentionControllerFrequencyInSeconds(0);
+    conf.setDeletedSegmentsRetentionInDays(0);
+    RetentionManager retentionManager =
+        new RetentionManager(pinotHelixResourceManager, leadControllerManager, conf, controllerMetrics);
+    retentionManager.start();
+    retentionManager.run();
+
+    SegmentDeletionManager deletionManager = pinotHelixResourceManager.getSegmentDeletionManager();
+
+    // Verify that the removeAgedDeletedSegments() method in deletion manager is actually called.
+    verify(deletionManager, times(1)).removeAgedDeletedSegments(leadControllerManager);
+
+    // Verify that the deleteSegments method is actually called.
+    verify(pinotHelixResourceManager, times(1)).deleteSegments(anyString(), anyList());

Review Comment:
   The deleted segments are checked in the setupPinotHelixResourceManager() as following, so I think this is not needed.
   ```
   // If and when PinotHelixResourceManager.deleteSegments() is invoked, make sure that the segments deleted
       // are exactly the same as the ones we expect to be deleted.
       doAnswer(new Answer() {
         @Override
         public Object answer(InvocationOnMock invocationOnMock)
             throws Throwable {
           Object[] args = invocationOnMock.getArguments();
           String tableNameArg = (String) args[0];
           Assert.assertEquals(tableNameArg, tableNameWithType);
           List<String> segmentListArg = (List<String>) args[1];
           Assert.assertEquals(segmentListArg.size(), removedSegments.size());
           for (String segmentName : removedSegments) {
             Assert.assertTrue(segmentListArg.contains(segmentName));
           }
           return null;
         }
       }).when(resourceManager).deleteSegments(anyString(), ArgumentMatchers.anyList());
   ```



-- 
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 #10754: Keep last completed segments for retention

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

   ## [Codecov](https://app.codecov.io/gh/apache/pinot/pull/10754?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 [#10754](https://app.codecov.io/gh/apache/pinot/pull/10754?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (e8ff5f8) into [master](https://app.codecov.io/gh/apache/pinot/commit/738e607106cf57adf8740691c458a17eb591ec56?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (738e607) will **decrease** coverage by `56.72%`.
   > The diff coverage is `68.75%`.
   
   ```diff
   @@              Coverage Diff              @@
   ##             master   #10754       +/-   ##
   =============================================
   - Coverage     70.47%   13.76%   -56.72%     
   + Complexity     6471      439     -6032     
   =============================================
     Files          2143     2089       -54     
     Lines        115192   112720     -2472     
     Branches      17354    17062      -292     
   =============================================
   - Hits          81180    15513    -65667     
   - Misses        28381    95935    +67554     
   + Partials       5631     1272     -4359     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration1 | `?` | |
   | integration2 | `?` | |
   | unittests1 | `?` | |
   | unittests2 | `13.76% <68.75%> (+<0.01%)` | :arrow_up: |
   
   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://app.codecov.io/gh/apache/pinot/pull/10754?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...troller/helix/core/retention/RetentionManager.java](https://app.codecov.io/gh/apache/pinot/pull/10754?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29udHJvbGxlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29udHJvbGxlci9oZWxpeC9jb3JlL3JldGVudGlvbi9SZXRlbnRpb25NYW5hZ2VyLmphdmE=) | `82.52% <33.33%> (+0.52%)` | :arrow_up: |
   | [...ntroller/helix/core/PinotHelixResourceManager.java](https://app.codecov.io/gh/apache/pinot/pull/10754?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29udHJvbGxlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29udHJvbGxlci9oZWxpeC9jb3JlL1Bpbm90SGVsaXhSZXNvdXJjZU1hbmFnZXIuamF2YQ==) | `63.39% <76.92%> (-8.02%)` | :arrow_down: |
   
   ... and [1692 files with indirect coverage changes](https://app.codecov.io/gh/apache/pinot/pull/10754/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] sajjad-moradi merged pull request #10754: Keep last completed segments for retention

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


-- 
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 #10754: Keep last completed segments for retention

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


##########
pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManager.java:
##########
@@ -872,6 +874,24 @@ public List<SegmentZKMetadata> getSegmentsZKMetadata(String tableNameWithType) {
     return ZKMetadataProvider.getSegmentsZKMetadata(_propertyStore, tableNameWithType);
   }
 
+  public List<String> getLastLLCCompletedSegments(String tableNameWithType) {
+    Map<Integer, String> partitionIdToLastLLCCompletedSegmentMap = new HashMap<>();
+    for (SegmentZKMetadata segMetadata : getSegmentsZKMetadata(tableNameWithType)) {
+      if (SegmentName.isLowLevelConsumerSegmentName(segMetadata.getSegmentName())
+          && segMetadata.getStatus() == CommonConstants.Segment.Realtime.Status.DONE) {
+        LLCSegmentName llcName = LLCSegmentName.of(segMetadata.getSegmentName());
+        int partitionGroupId = llcName.getPartitionGroupId();
+        int sequenceNumber = llcName.getSequenceNumber();
+        String lastCompletedSegName = partitionIdToLastLLCCompletedSegmentMap.get(partitionGroupId);
+        if (lastCompletedSegName == null
+            || LLCSegmentName.of(lastCompletedSegName).getSequenceNumber() < sequenceNumber) {
+          partitionIdToLastLLCCompletedSegmentMap.put(partitionGroupId, segMetadata.getSegmentName());
+        }
+      }
+    }
+    return new ArrayList<>(partitionIdToLastLLCCompletedSegmentMap.values());

Review Comment:
   you can just return `partitionIdToLastLLCCompletedSegmentMap.values()`, no need to wrap with another array list.



##########
pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/PinotHelixResourceManager.java:
##########
@@ -872,6 +874,24 @@ public List<SegmentZKMetadata> getSegmentsZKMetadata(String tableNameWithType) {
     return ZKMetadataProvider.getSegmentsZKMetadata(_propertyStore, tableNameWithType);
   }
 
+  public List<String> getLastLLCCompletedSegments(String tableNameWithType) {
+    Map<Integer, String> partitionIdToLastLLCCompletedSegmentMap = new HashMap<>();
+    for (SegmentZKMetadata segMetadata : getSegmentsZKMetadata(tableNameWithType)) {

Review Comment:
   nit: use `ZKMetadataProvider.getLLCRealtimeSegments()` if you want to just get the metadata for realtime segments.



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