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/11/23 07:49:35 UTC

[GitHub] [pinot] navina opened a new pull request, #9851: Merge new columns in existing record with default merge strategy

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

   Related to #9771 
   
   After a schema evolves (eg. new column added), the column is not treated as an upsert column by the partition upsert manager because it is not part of `PartialUpsertHandler#column2Mergers`. This PR attempts to fix this by iterating through all columns in the record and applying a default merge strategy on all columns, except primary key and comparison column. 
   
   Note: This PR can handle schema evolution of adding new columns without restarting the server, as long as there are no changes to the upsert config. Any change made to the `upsertConfig: {}`  section in the table config of a partial upsert enabled table will require a server restart to correctly reload all segments and the metadata. 
   
   Labels: `bugfix` 
   


-- 
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 #9851: Merge new columns in existing record with default merge strategy

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


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/upsert/PartialUpsertHandler.java:
##########
@@ -33,18 +34,25 @@
 public class PartialUpsertHandler {
   // _column2Mergers maintains the mapping of merge strategies per columns.
   private final Map<String, PartialUpsertMerger> _column2Mergers = new HashMap<>();
+  private final UpsertConfig.Strategy _defaultPartialUpsertStrategy;

Review Comment:
   `PartialUpsertMergerFactory.getMerger` actually returns the same singleton instance. So, there is only one merger instance created. 
   But I can make this change as it get rids of the assumption in the underlying implementation. 



-- 
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 #9851: Merge new columns in existing record with default merge strategy

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


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/upsert/PartialUpsertHandler.java:
##########
@@ -33,18 +34,25 @@
 public class PartialUpsertHandler {
   // _column2Mergers maintains the mapping of merge strategies per columns.
   private final Map<String, PartialUpsertMerger> _column2Mergers = new HashMap<>();
+  private final UpsertConfig.Strategy _defaultPartialUpsertStrategy;
+  private final String _comparisonColumn;
+  private final List<String> _primaryKeyColumns;
 
   public PartialUpsertHandler(Schema schema, Map<String, UpsertConfig.Strategy> partialUpsertStrategies,
       UpsertConfig.Strategy defaultPartialUpsertStrategy, String comparisonColumn) {
+    _defaultPartialUpsertStrategy = defaultPartialUpsertStrategy;
+    _comparisonColumn = comparisonColumn;
+    _primaryKeyColumns = schema.getPrimaryKeyColumns();
+
     for (Map.Entry<String, UpsertConfig.Strategy> entry : partialUpsertStrategies.entrySet()) {
       _column2Mergers.put(entry.getKey(), PartialUpsertMergerFactory.getMerger(entry.getValue()));
     }
     // For all physical columns (including date time columns) except for primary key columns and comparison column.
     // If no comparison column is configured, use main time column as the comparison time.
     for (String columnName : schema.getPhysicalColumnNames()) {
-      if (!schema.getPrimaryKeyColumns().contains(columnName) && !_column2Mergers.containsKey(columnName)
-          && !comparisonColumn.equals(columnName)) {
-        _column2Mergers.put(columnName, PartialUpsertMergerFactory.getMerger(defaultPartialUpsertStrategy));
+      if (!_primaryKeyColumns.contains(columnName) && !_column2Mergers.containsKey(columnName)
+          && !_comparisonColumn.equals(columnName)) {
+        _column2Mergers.put(columnName, PartialUpsertMergerFactory.getMerger(_defaultPartialUpsertStrategy));
       }
     }

Review Comment:
   ok yeah this part is redundant now. 



-- 
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 #9851: Merge new columns in existing record with default merge strategy

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


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/upsert/PartialUpsertHandler.java:
##########
@@ -65,15 +73,18 @@ public PartialUpsertHandler(Schema schema, Map<String, UpsertConfig.Strategy> pa
    * @return a new row after merge
    */
   public GenericRow merge(GenericRow previousRecord, GenericRow newRecord) {
-    for (Map.Entry<String, PartialUpsertMerger> entry : _column2Mergers.entrySet()) {
-      String column = entry.getKey();
-      if (!previousRecord.isNullValue(column)) {
-        if (newRecord.isNullValue(column)) {
-          newRecord.putValue(column, previousRecord.getValue(column));
-          newRecord.removeNullValueField(column);
-        } else {
-          newRecord.putValue(column,
-              entry.getValue().merge(previousRecord.getValue(column), newRecord.getValue(column)));
+    for (String column : previousRecord.getFieldToValueMap().keySet()) {

Review Comment:
   the new column will always be present in `newRecord` (which is equivalent to an overwrite strategy). if there is an update to the same primary key, then the previous record will have that new column value and iterate through this loop. So, it should be fine to iterate through the columns of the previous record, right? 



-- 
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 #9851: Merge new columns in existing record with default merge strategy

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

   # [Codecov](https://codecov.io/gh/apache/pinot/pull/9851?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 [#9851](https://codecov.io/gh/apache/pinot/pull/9851?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (9c73dd8) into [master](https://codecov.io/gh/apache/pinot/commit/d62a867d46433567f7e2c488745013a6a847e4ab?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (d62a867) will **decrease** coverage by `2.64%`.
   > The diff coverage is `93.75%`.
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #9851      +/-   ##
   ============================================
   - Coverage     70.36%   67.72%   -2.65%     
   + Complexity     5469     5267     -202     
   ============================================
     Files          1972     1468     -504     
     Lines        105675    76913   -28762     
     Branches      15989    12230    -3759     
   ============================================
   - Hits          74361    52088   -22273     
   + Misses        26116    21135    -4981     
   + Partials       5198     3690    -1508     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration1 | `?` | |
   | integration2 | `?` | |
   | unittests1 | `67.72% <93.75%> (-0.03%)` | :arrow_down: |
   | 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/9851?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...not/segment/local/upsert/PartialUpsertHandler.java](https://codecov.io/gh/apache/pinot/pull/9851/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC91cHNlcnQvUGFydGlhbFVwc2VydEhhbmRsZXIuamF2YQ==) | `96.15% <93.75%> (-3.85%)` | :arrow_down: |
   | [...va/org/apache/pinot/common/config/NettyConfig.java](https://codecov.io/gh/apache/pinot/pull/9851/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/9851/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/9851/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/9851/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/9851/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: |
   | [...ache/pinot/server/access/AccessControlFactory.java](https://codecov.io/gh/apache/pinot/pull/9851/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VydmVyL2FjY2Vzcy9BY2Nlc3NDb250cm9sRmFjdG9yeS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...he/pinot/common/messages/SegmentReloadMessage.java](https://codecov.io/gh/apache/pinot/pull/9851/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-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWVzc2FnZXMvU2VnbWVudFJlbG9hZE1lc3NhZ2UuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...he/pinot/common/messages/TableDeletionMessage.java](https://codecov.io/gh/apache/pinot/pull/9851/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-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWVzc2FnZXMvVGFibGVEZWxldGlvbk1lc3NhZ2UuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...pinot/core/data/manager/realtime/TimerService.java](https://codecov.io/gh/apache/pinot/pull/9851/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9kYXRhL21hbmFnZXIvcmVhbHRpbWUvVGltZXJTZXJ2aWNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | ... and [745 more](https://codecov.io/gh/apache/pinot/pull/9851/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] Jackie-Jiang commented on a diff in pull request #9851: Merge new columns in existing record with default merge strategy

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


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/upsert/PartialUpsertHandler.java:
##########
@@ -33,18 +34,25 @@
 public class PartialUpsertHandler {
   // _column2Mergers maintains the mapping of merge strategies per columns.
   private final Map<String, PartialUpsertMerger> _column2Mergers = new HashMap<>();
+  private final UpsertConfig.Strategy _defaultPartialUpsertStrategy;

Review Comment:
   Store a default merger (`PartialUpsertMergerFactory.getMerger(defaultPartialUpsertStrategy)`) instead of the strategy to avoid per record creation of the merger



##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/upsert/PartialUpsertHandler.java:
##########
@@ -33,18 +34,25 @@
 public class PartialUpsertHandler {
   // _column2Mergers maintains the mapping of merge strategies per columns.
   private final Map<String, PartialUpsertMerger> _column2Mergers = new HashMap<>();
+  private final UpsertConfig.Strategy _defaultPartialUpsertStrategy;
+  private final String _comparisonColumn;
+  private final List<String> _primaryKeyColumns;
 
   public PartialUpsertHandler(Schema schema, Map<String, UpsertConfig.Strategy> partialUpsertStrategies,
       UpsertConfig.Strategy defaultPartialUpsertStrategy, String comparisonColumn) {
+    _defaultPartialUpsertStrategy = defaultPartialUpsertStrategy;
+    _comparisonColumn = comparisonColumn;
+    _primaryKeyColumns = schema.getPrimaryKeyColumns();
+
     for (Map.Entry<String, UpsertConfig.Strategy> entry : partialUpsertStrategies.entrySet()) {
       _column2Mergers.put(entry.getKey(), PartialUpsertMergerFactory.getMerger(entry.getValue()));
     }
     // For all physical columns (including date time columns) except for primary key columns and comparison column.
     // If no comparison column is configured, use main time column as the comparison time.
     for (String columnName : schema.getPhysicalColumnNames()) {
-      if (!schema.getPrimaryKeyColumns().contains(columnName) && !_column2Mergers.containsKey(columnName)
-          && !comparisonColumn.equals(columnName)) {
-        _column2Mergers.put(columnName, PartialUpsertMergerFactory.getMerger(defaultPartialUpsertStrategy));
+      if (!_primaryKeyColumns.contains(columnName) && !_column2Mergers.containsKey(columnName)
+          && !_comparisonColumn.equals(columnName)) {
+        _column2Mergers.put(columnName, PartialUpsertMergerFactory.getMerger(_defaultPartialUpsertStrategy));
       }
     }

Review Comment:
   This part is no longer needed. We apply default strategy for all columns not explicitly configured



-- 
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 #9851: Merge new columns in existing record with default merge strategy

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

   @Jackie-Jiang / @KKcorps : 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] KKcorps commented on a diff in pull request #9851: Merge new columns in existing record with default merge strategy

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


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/upsert/PartialUpsertHandler.java:
##########
@@ -65,15 +73,18 @@ public PartialUpsertHandler(Schema schema, Map<String, UpsertConfig.Strategy> pa
    * @return a new row after merge
    */
   public GenericRow merge(GenericRow previousRecord, GenericRow newRecord) {
-    for (Map.Entry<String, PartialUpsertMerger> entry : _column2Mergers.entrySet()) {
-      String column = entry.getKey();
-      if (!previousRecord.isNullValue(column)) {
-        if (newRecord.isNullValue(column)) {
-          newRecord.putValue(column, previousRecord.getValue(column));
-          newRecord.removeNullValueField(column);
-        } else {
-          newRecord.putValue(column,
-              entry.getValue().merge(previousRecord.getValue(column), newRecord.getValue(column)));
+    for (String column : previousRecord.getFieldToValueMap().keySet()) {

Review Comment:
   Makes sense.



-- 
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 #9851: Merge new columns in existing record with default merge strategy

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


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/upsert/PartialUpsertHandler.java:
##########
@@ -65,15 +73,18 @@ public PartialUpsertHandler(Schema schema, Map<String, UpsertConfig.Strategy> pa
    * @return a new row after merge
    */
   public GenericRow merge(GenericRow previousRecord, GenericRow newRecord) {
-    for (Map.Entry<String, PartialUpsertMerger> entry : _column2Mergers.entrySet()) {
-      String column = entry.getKey();
-      if (!previousRecord.isNullValue(column)) {
-        if (newRecord.isNullValue(column)) {
-          newRecord.putValue(column, previousRecord.getValue(column));
-          newRecord.removeNullValueField(column);
-        } else {
-          newRecord.putValue(column,
-              entry.getValue().merge(previousRecord.getValue(column), newRecord.getValue(column)));
+    for (String column : previousRecord.getFieldToValueMap().keySet()) {

Review Comment:
   shouldn't the iteration be done on the columns in the newRecord?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] Jackie-Jiang merged pull request #9851: Merge new columns in existing record with default merge strategy

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang merged PR #9851:
URL: https://github.com/apache/pinot/pull/9851


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