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/04/18 18:43:09 UTC

[GitHub] [pinot] navina opened a new pull request, #8564: Invoke checkpoint before closing partition group consumer

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

   Required for consumers that rely on `checkpoint` API to update consumption status at source. Otherwise, when the stream is idle for too long, the `LLRealtimeSegmentDataManager` would simply close the consumer and restart from the initial `startOffset`. This can cause duplicates in such consumers. 


-- 
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 commented on a diff in pull request #8564: Invoke checkpoint before closing partition group consumer

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on code in PR #8564:
URL: https://github.com/apache/pinot/pull/8564#discussion_r854416554


##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/LLRealtimeSegmentDataManager.java:
##########
@@ -1333,12 +1334,12 @@ public LLRealtimeSegmentDataManager(SegmentZKMetadata segmentZKMetadata, TableCo
     }
 
     try {
+      _startOffset = _streamPartitionMsgOffsetFactory.create(_segmentZKMetadata.getStartOffset());

Review Comment:
   Can be simplified to `_partitionGroupConsumptionStatus.getStartOffset()`



##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/LLRealtimeSegmentDataManager.java:
##########
@@ -1457,7 +1458,16 @@ private void makeStreamConsumer(String reason) {
     _segmentLogger.info("Creating new stream consumer for topic partition {} , reason: {}", _clientId, reason);
     _partitionGroupConsumer =
         _streamConsumerFactory.createPartitionGroupConsumer(_clientId, _partitionGroupConsumptionStatus);
-    _partitionGroupConsumer.start(_partitionGroupConsumptionStatus.getStartOffset());
+    _partitionGroupConsumer.start(_currentOffset);
+  }
+  /**
+   * Checkpoints existing consumer before creating a new consumer instance
+   */
+  private void recreateStreamConsumer(String reason) {
+    if (_partitionGroupConsumer != null) {
+      _currentOffset = _partitionGroupConsumer.checkpoint(_currentOffset);
+    }
+    makeStreamConsumer(reason);

Review Comment:
   Let's not reuse the `makeStreamConsumer()` for clear separation of the handling
   ```suggestion
       _segmentLogger.warn("Recreating stream consumer for topic partition: {}, reason: {}", _clientId, reason);
       _currentOffset = _partitionGroupConsumer.checkpoint(_currentOffset);
       closePartitionGroupConsumer();
       _partitionGroupConsumer =
           _streamConsumerFactory.createPartitionGroupConsumer(_clientId, _partitionGroupConsumptionStatus);
       _partitionGroupConsumer.start(_currentOffset);
   ```



##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/LLRealtimeSegmentDataManager.java:
##########
@@ -1457,7 +1458,16 @@ private void makeStreamConsumer(String reason) {
     _segmentLogger.info("Creating new stream consumer for topic partition {} , reason: {}", _clientId, reason);
     _partitionGroupConsumer =
         _streamConsumerFactory.createPartitionGroupConsumer(_clientId, _partitionGroupConsumptionStatus);
-    _partitionGroupConsumer.start(_partitionGroupConsumptionStatus.getStartOffset());
+    _partitionGroupConsumer.start(_currentOffset);
+  }

Review Comment:
   (minor) Add a new line



##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/LLRealtimeSegmentDataManager.java:
##########
@@ -978,6 +978,7 @@ private void closeStreamConsumers() {
 
   private void closePartitionGroupConsumer() {
     try {
+//      _currentOffset = _partitionGroupConsumer.checkpoint(_currentOffset);

Review Comment:
   (minor) remove



-- 
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 #8564: Invoke checkpoint before closing partition group consumer

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


##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/LLRealtimeSegmentDataManager.java:
##########
@@ -1333,12 +1334,12 @@ public LLRealtimeSegmentDataManager(SegmentZKMetadata segmentZKMetadata, TableCo
     }
 
     try {
+      _startOffset = _streamPartitionMsgOffsetFactory.create(_segmentZKMetadata.getStartOffset());

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] npawar commented on pull request #8564: Invoke checkpoint before closing partition group consumer

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

   In the case you mention @Jackie-Jiang , after reset, the new consuming segment will pick offset from the "endOffset" field in previous segment's zk metadata. Like Navina said, it will cause a checkpoint at source, but won't be used.


-- 
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 #8564: Invoke checkpoint before closing partition group consumer

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

   @npawar yep. I clarified it over a call with Jackie. He did suggest a different fix that will not overload the "close" semantics in `LLRealtimeSegmentDataManager`. Basically, we can create a method for "recreating" stream consumer in this class. In that method, we can always invoke checkpoint and update the currentOffset. I will update the PR tonight. I think that will solve the problem without overloading the close method. 
   


-- 
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 commented on pull request #8564: Invoke checkpoint before closing partition group consumer

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on PR #8564:
URL: https://github.com/apache/pinot/pull/8564#issuecomment-1103238294

   I don't follow the fix here. Let's say a consuming segment is dropped (or reset), we will destroy the current consuming segment, and that will add a checkpoint to the source. The next consuming segment will skip the records within the dropped segment, and cause data loss.
   IMO we should add the checkpoint only when then previous segment is successfully sealed and we want to start a new consuming 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] navina commented on pull request #8564: Invoke checkpoint before closing partition group consumer

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

   >  Let's say a consuming segment is dropped (or reset), we will destroy the current consuming segment, and that will add a checkpoint to the source. The next consuming segment will skip the records within the dropped segment, and cause data loss.
   
   If you reset or drop a consuming segment, it is ok to destroy the consuming segment. It may cause a checkpoint at source. But your reset point should still pick up from where you actually "reset". Why would it skip the records within dropped segment ? I don't follow this part of your argument. 
   Is it in the context of kafka like systems where the offsets are guaranteed to be simple monotonically increasing integers? Can you explain a bit? perhaps with kafka as a context?  
   
   


-- 
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 #8564: Invoke checkpoint before closing partition group consumer

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

   > I think we should strongly consider updating the interface so that the close() can do whatever the underlying stream wants to do. If that means checkpointing in the middle of an idle stream, so be it. You can add another method to the interface whose default implementation is to call the prev close. You should not need to modify any other stream that way
   
   @mcvsubbu This actually feels like an undiscovered bug for idle streams / errored streams due to the `LLRealtimeSegmentDataManager` constantly closes and recreates the consumer.
   Having said, Jackie made a good point about not overloading the semantics of close. If `LLRealtimeSegmentDataManager` needs full control over the consumer, then it is better to be explicit about invoking the `checkpoint` method. That is the contract made with `PartitionGroupConsumer`. I really don't think an interface change is needed for a bug fix. It will be a non-trivial change. 


-- 
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 #8564: Invoke checkpoint before closing partition group consumer

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


##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/LLRealtimeSegmentDataManager.java:
##########
@@ -978,6 +978,7 @@ private void closeStreamConsumers() {
 
   private void closePartitionGroupConsumer() {
     try {
+      _currentOffset = _partitionGroupConsumer.checkpoint(_currentOffset);

Review Comment:
   1. I would like to avoid interface changes for bugfixes. This is a bug that is valid only after the introduction of checkpoint interface.
   2. It is not just the underlying streams that need to handle this. The LLRealtimeSegmentDataManager should also use the current offset when creating a new consumer and not always the startOffset. 
   



-- 
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 #8564: Invoke checkpoint before closing partition group consumer

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

   > lgtm! Is there ay scope to add a unit test to check that the partitionGroupConsumer starts with the right startOffset across closes?
   
   I took a quick look. It is using a lot of mock/fake classes and based on my understanding, it is not possible to test this behavior easily. :/ 


-- 
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 merged pull request #8564: Invoke checkpoint before closing partition group consumer

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


-- 
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] mcvsubbu commented on a diff in pull request #8564: Invoke checkpoint before closing partition group consumer

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


##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/LLRealtimeSegmentDataManager.java:
##########
@@ -978,6 +978,7 @@ private void closeStreamConsumers() {
 
   private void closePartitionGroupConsumer() {
     try {
+      _currentOffset = _partitionGroupConsumer.checkpoint(_currentOffset);

Review Comment:
   Why not modify the interface to have a `close(offset)` call ? that way, underlying streams can do whatever they want with the offset.
   
   Also, it mirrors how the consumer is constructed with the start offset



-- 
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 #8564: Invoke checkpoint before closing partition group consumer

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

   # [Codecov](https://codecov.io/gh/apache/pinot/pull/8564?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > :exclamation: No coverage uploaded for pull request base (`master@d503d50`). [Click here to learn what that means](https://docs.codecov.io/docs/error-reference?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#section-missing-base-commit).
   > The diff coverage is `0.00%`.
   
   ```diff
   @@            Coverage Diff            @@
   ##             master    #8564   +/-   ##
   =========================================
     Coverage          ?   14.15%           
     Complexity        ?       84           
   =========================================
     Files             ?     1641           
     Lines             ?    86308           
     Branches          ?    13142           
   =========================================
     Hits              ?    12214           
     Misses            ?    73179           
     Partials          ?      915           
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | unittests2 | `14.15% <0.00%> (?)` | |
   
   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/8564?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/LLRealtimeSegmentDataManager.java](https://codecov.io/gh/apache/pinot/pull/8564/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) | `0.00% <0.00%> (ø)` | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/pinot/pull/8564?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/pinot/pull/8564?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [d503d50...4001749](https://codecov.io/gh/apache/pinot/pull/8564?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?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] Jackie-Jiang commented on a diff in pull request #8564: Invoke checkpoint before closing partition group consumer

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on code in PR #8564:
URL: https://github.com/apache/pinot/pull/8564#discussion_r854432032


##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/LLRealtimeSegmentDataManager.java:
##########
@@ -1457,7 +1457,21 @@ private void makeStreamConsumer(String reason) {
     _segmentLogger.info("Creating new stream consumer for topic partition {} , reason: {}", _clientId, reason);
     _partitionGroupConsumer =
         _streamConsumerFactory.createPartitionGroupConsumer(_clientId, _partitionGroupConsumptionStatus);
-    _partitionGroupConsumer.start(_partitionGroupConsumptionStatus.getStartOffset());
+    _partitionGroupConsumer.start(_currentOffset);
+  }
+
+  /**
+   * Checkpoints existing consumer before creating a new consumer instance
+   */
+  private void recreateStreamConsumer(String reason) {
+    _segmentLogger.warn("Recreating stream consumer for topic partition {}, reason: {}", _clientId, reason);
+    if (_partitionGroupConsumer != null) {

Review Comment:
   (minor) This check and the check on line 1454 can be removed because `_partitionGroupConsumer` is always `null` in `makeStreamConsumer()`, and always not `null` in `recreateStreamConsumer()`



-- 
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] mcvsubbu commented on pull request #8564: Invoke checkpoint before closing partition group consumer

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

   > 
   
   I think we should strongly consider updating the interface so that the close() can do whatever the underlying stream wants to do. If that means checkpointing in the middle of an idle stream, so be 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] navina commented on a diff in pull request #8564: Invoke checkpoint before closing partition group consumer

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


##########
pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/LLRealtimeSegmentDataManager.java:
##########
@@ -1457,7 +1458,16 @@ private void makeStreamConsumer(String reason) {
     _segmentLogger.info("Creating new stream consumer for topic partition {} , reason: {}", _clientId, reason);
     _partitionGroupConsumer =
         _streamConsumerFactory.createPartitionGroupConsumer(_clientId, _partitionGroupConsumptionStatus);
-    _partitionGroupConsumer.start(_partitionGroupConsumptionStatus.getStartOffset());
+    _partitionGroupConsumer.start(_currentOffset);
+  }
+  /**
+   * Checkpoints existing consumer before creating a new consumer instance
+   */
+  private void recreateStreamConsumer(String reason) {
+    if (_partitionGroupConsumer != null) {
+      _currentOffset = _partitionGroupConsumer.checkpoint(_currentOffset);
+    }
+    makeStreamConsumer(reason);

Review Comment:
   Ok. I have made the change. 



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