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 2021/05/10 19:48:59 UTC

[GitHub] [incubator-pinot] deemoliu opened a new pull request #6899: Add partial upsert config and mergers (WIP)

deemoliu opened a new pull request #6899:
URL: https://github.com/apache/incubator-pinot/pull/6899


   ## Description
   <!-- Add a description of your PR here.
   A good description should include pointers to an issue or design document, etc.
   -->
   ## Upgrade Notes
   Does this PR prevent a zero down-time upgrade? (Assume upgrade order: Controller, Broker, Server, Minion)
   * [ ] Yes (Please label as **<code>backward-incompat</code>**, and complete the section below on Release Notes)
   
   Does this PR fix a zero-downtime upgrade introduced earlier?
   * [ ] Yes (Please label this as **<code>backward-incompat</code>**, and complete the section below on Release Notes)
   
   Does this PR otherwise need attention when creating release notes? Things to consider:
   - New configuration options
   - Deprecation of configurations
   - Signature changes to public methods/interfaces
   - New plugins added or old plugins removed
   * [ ] Yes (Please label this PR as **<code>release-notes</code>** and complete the section on Release Notes)
   ## Release Notes
   <!-- If you have tagged this as either backward-incompat or release-notes,
   you MUST add text here that you would like to see appear in release notes of the
   next release. -->
   
   <!-- If you have a series of commits adding or enabling a feature, then
   add this section only in final commit that marks the feature completed.
   Refer to earlier release notes to see examples of text.
   -->
   ## Documentation
   <!-- If you have introduced a new feature or configuration, please add it to the documentation as well.
   See https://docs.pinot.apache.org/developers/developers-and-contributors/update-document
   -->
   


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

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



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


[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #6899: Add partial upsert config and mergers

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on a change in pull request #6899:
URL: https://github.com/apache/incubator-pinot/pull/6899#discussion_r651107004



##########
File path: pinot-segment-local/src/main/java/org/apache/pinot/segment/local/upsert/PartitionUpsertMetadataManager.java
##########
@@ -60,128 +64,175 @@
   private final String _tableNameWithType;
   private final int _partitionId;
   private final ServerMetrics _serverMetrics;
+  private final PartialUpsertHandler _partialUpsertHandler;
 
-  public PartitionUpsertMetadataManager(String tableNameWithType, int partitionId, ServerMetrics serverMetrics) {
+  // TODO(upset): consider an off-heap KV store to persist this index to improve the recovery speed.
+  @VisibleForTesting
+  final ConcurrentHashMap<PrimaryKey, RecordLocation> _primaryKeyToRecordLocationMap = new ConcurrentHashMap<>();
+
+  // Reused for reading previous record during partial upsert
+  private final GenericRow _reuse = new GenericRow();
+  // Stores the result of updateRecord()
+  private GenericRow _result;
+
+  public PartitionUpsertMetadataManager(String tableNameWithType, int partitionId, ServerMetrics serverMetrics,
+      @Nullable PartialUpsertHandler partialUpsertHandler) {
     _tableNameWithType = tableNameWithType;
     _partitionId = partitionId;
     _serverMetrics = serverMetrics;
+    _partialUpsertHandler = partialUpsertHandler;
   }
 
-  public ConcurrentHashMap<PrimaryKey, RecordLocation> getPrimaryKeyToRecordLocationMap() {
-    return _primaryKeyToRecordLocationMap;
+  /**
+   * Initializes the upsert metadata for the given immutable segment.
+   */
+  public void addSegment(ImmutableSegmentImpl immutableSegment, Iterator<RecordInfo> recordInfoIterator) {
+    addSegment(immutableSegment, recordInfoIterator, new ThreadSafeMutableRoaringBitmap());
   }
 
-  // TODO(upset): consider an off-heap KV store to persist this index to improve the recovery speed.
   @VisibleForTesting
-  final ConcurrentHashMap<PrimaryKey, RecordLocation> _primaryKeyToRecordLocationMap = new ConcurrentHashMap<>();
-
-  /**
-   * Initializes the upsert metadata for the given immutable segment, returns the valid doc ids for the segment.
-   */
-  public ThreadSafeMutableRoaringBitmap addSegment(String segmentName, Iterator<RecordInfo> recordInfoIterator) {
+  void addSegment(ImmutableSegmentImpl immutableSegment, Iterator<RecordInfo> recordInfoIterator,
+      ThreadSafeMutableRoaringBitmap validDocIds) {
+    String segmentName = immutableSegment.getSegmentName();
     LOGGER.info("Adding upsert metadata for segment: {}", segmentName);
+    immutableSegment.enableUpsert(this, validDocIds);
 
-    ThreadSafeMutableRoaringBitmap validDocIds = new ThreadSafeMutableRoaringBitmap();
     while (recordInfoIterator.hasNext()) {
       RecordInfo recordInfo = recordInfoIterator.next();
       _primaryKeyToRecordLocationMap.compute(recordInfo._primaryKey, (primaryKey, currentRecordLocation) -> {
         if (currentRecordLocation != null) {
           // Existing primary key
 
-          if (segmentName.equals(currentRecordLocation.getSegmentName())) {
-            // The current record location has the same segment name
-
-            // Update the record location when the new timestamp is greater than or equal to the current timestamp.
-            // There are 2 scenarios:
-            //   1. The current record location is pointing to the same segment (the segment being added). In this case,
-            //      we want to update the record location when there is a tie to keep the newer record. Note that the
-            //      record info iterator will return records with incremental doc ids.
-            //   2. The current record location is pointing to the old segment being replaced. This could happen when
-            //      committing a consuming segment, or reloading a completed segment. In this case, we want to update
-            //      the record location when there is a tie because the record locations should point to the new added
-            //      segment instead of the old segment being replaced. Also, do not update the valid doc ids for the old
-            //      segment because it has not been replaced yet.
+          // The current record is in the same segment
+          // Update the record location when there is a tie to keep the newer record. Note that the record info iterator
+          // will return records with incremental doc ids.
+          IndexSegment currentSegment = currentRecordLocation.getSegment();
+          if (immutableSegment == currentSegment) {
             if (recordInfo._timestamp >= currentRecordLocation.getTimestamp()) {
-              // Only update the valid doc ids for the new segment
-              if (validDocIds == currentRecordLocation.getValidDocIds()) {
-                validDocIds.remove(currentRecordLocation.getDocId());
-              }
+              validDocIds.remove(currentRecordLocation.getDocId());
               validDocIds.add(recordInfo._docId);
-              return new RecordLocation(segmentName, recordInfo._docId, recordInfo._timestamp, validDocIds);
+              return new RecordLocation(immutableSegment, recordInfo._docId, recordInfo._timestamp);
             } else {
               return currentRecordLocation;
             }
-          } else {
-            // The current record location is pointing to a different segment
-
-            // Update the record location when getting a newer timestamp, or the timestamp is the same as the current
-            // timestamp, but the segment has a larger sequence number (the segment is newer than the current segment).
-            if (recordInfo._timestamp > currentRecordLocation.getTimestamp() || (
-                recordInfo._timestamp == currentRecordLocation.getTimestamp()
-                    && LLCSegmentName.isLowLevelConsumerSegmentName(segmentName)
-                    && LLCSegmentName.isLowLevelConsumerSegmentName(currentRecordLocation.getSegmentName())
-                    && LLCSegmentName.getSequenceNumber(segmentName) > LLCSegmentName
-                    .getSequenceNumber(currentRecordLocation.getSegmentName()))) {
-              currentRecordLocation.getValidDocIds().remove(currentRecordLocation.getDocId());
+          }
+
+          // The current record is in an old segment being replaced
+          // This could happen when committing a consuming segment, or reloading a completed segment. In this case, we
+          // want to update the record location when there is a tie because the record locations should point to the new
+          // added segment instead of the old segment being replaced. Also, do not update the valid doc ids for the old
+          // segment because it has not been replaced yet.
+          String currentSegmentName = currentSegment.getSegmentName();
+          if (segmentName.equals(currentSegmentName)) {
+            if (recordInfo._timestamp >= currentRecordLocation.getTimestamp()) {
               validDocIds.add(recordInfo._docId);
-              return new RecordLocation(segmentName, recordInfo._docId, recordInfo._timestamp, validDocIds);
+              return new RecordLocation(immutableSegment, recordInfo._docId, recordInfo._timestamp);
             } else {
               return currentRecordLocation;
             }
           }
+
+          // The current record is in a different segment
+          // Update the record location when getting a newer timestamp, or the timestamp is the same as the current
+          // timestamp, but the segment has a larger sequence number (the segment is newer than the current segment).
+          if (recordInfo._timestamp > currentRecordLocation.getTimestamp() || (
+              recordInfo._timestamp == currentRecordLocation.getTimestamp() && LLCSegmentName
+                  .isLowLevelConsumerSegmentName(segmentName) && LLCSegmentName
+                  .isLowLevelConsumerSegmentName(currentSegmentName)
+                  && LLCSegmentName.getSequenceNumber(segmentName) > LLCSegmentName
+                  .getSequenceNumber(currentSegmentName))) {
+            assert currentSegment.getValidDocIds() != null;
+            currentSegment.getValidDocIds().remove(currentRecordLocation.getDocId());
+            validDocIds.add(recordInfo._docId);
+            return new RecordLocation(immutableSegment, recordInfo._docId, recordInfo._timestamp);
+          } else {
+            return currentRecordLocation;
+          }
         } else {
           // New primary key
           validDocIds.add(recordInfo._docId);
-          return new RecordLocation(segmentName, recordInfo._docId, recordInfo._timestamp, validDocIds);
+          return new RecordLocation(immutableSegment, recordInfo._docId, recordInfo._timestamp);
         }
       });
     }
     // Update metrics
     _serverMetrics.setValueOfPartitionGauge(_tableNameWithType, _partitionId, ServerGauge.UPSERT_PRIMARY_KEYS_COUNT,
         _primaryKeyToRecordLocationMap.size());
-    return validDocIds;
   }
 
   /**
-   * Updates the upsert metadata for a new consumed record in the given consuming segment.
+   * Updates the upsert metadata for a new consumed record in the given consuming segment. Returns the merged record if
+   * partial-upsert is enabled.
    */
-  public void updateRecord(String segmentName, RecordInfo recordInfo, ThreadSafeMutableRoaringBitmap validDocIds) {
+  public GenericRow updateRecord(IndexSegment segment, RecordInfo recordInfo, GenericRow record) {
+    // For partial-upsert, need to ensure all previous records are loaded before inserting new records.
+    if (_partialUpsertHandler != null) {
+      while (!_partialUpsertHandler.isAllSegmentsLoaded()) {
+        LOGGER
+            .info("Sleeping 1 second waiting for all segments loaded for partial-upsert table: {}", _tableNameWithType);
+        try {
+          //noinspection BusyWait
+          Thread.sleep(1000L);
+        } catch (InterruptedException e) {
+          throw new RuntimeException(e);
+        }
+      }
+    }
+
+    _result = record;
     _primaryKeyToRecordLocationMap.compute(recordInfo._primaryKey, (primaryKey, currentRecordLocation) -> {
       if (currentRecordLocation != null) {
         // Existing primary key
 
         // Update the record location when the new timestamp is greater than or equal to the current timestamp. Update
         // the record location when there is a tie to keep the newer record.
         if (recordInfo._timestamp >= currentRecordLocation.getTimestamp()) {
-          currentRecordLocation.getValidDocIds().remove(currentRecordLocation.getDocId());
-          validDocIds.add(recordInfo._docId);
-          return new RecordLocation(segmentName, recordInfo._docId, recordInfo._timestamp, validDocIds);
+          IndexSegment currentSegment = currentRecordLocation.getSegment();
+          if (_partialUpsertHandler != null) {
+            // Partial upsert
+            GenericRow previousRecord = currentSegment.getRecord(currentRecordLocation.getDocId(), _reuse);

Review comment:
       The entry will be removed before destroying the segment (`removeSegment()` will be called)




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

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



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


[GitHub] [incubator-pinot] codecov-commenter edited a comment on pull request #6899: Add partial upsert config and mergers

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #6899:
URL: https://github.com/apache/incubator-pinot/pull/6899#issuecomment-853568285


   # [Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6899?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 [#6899](https://codecov.io/gh/apache/incubator-pinot/pull/6899?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (e49861f) into [master](https://codecov.io/gh/apache/incubator-pinot/commit/282edba94d63ddbaabc8bf2cfc55030bf1c1f30d?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (282edba) will **decrease** coverage by `31.82%`.
   > The diff coverage is `30.18%`.
   
   > :exclamation: Current head e49861f differs from pull request most recent head c54b86f. Consider uploading reports for the commit c54b86f to get more accurate results
   [![Impacted file tree graph](https://codecov.io/gh/apache/incubator-pinot/pull/6899/graphs/tree.svg?width=650&height=150&src=pr&token=4ibza2ugkz&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/incubator-pinot/pull/6899?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@              Coverage Diff              @@
   ##             master    #6899       +/-   ##
   =============================================
   - Coverage     73.62%   41.80%   -31.83%     
   + Complexity       91        7       -84     
   =============================================
     Files          1482     1486        +4     
     Lines         72923    73071      +148     
     Branches      10493    10526       +33     
   =============================================
   - Hits          53693    30548    -23145     
   - Misses        15756    39952    +24196     
   + Partials       3474     2571      -903     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration | `41.80% <30.18%> (+0.03%)` | :arrow_up: |
   | unittests | `?` | |
   
   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/incubator-pinot/pull/6899?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...core/query/pruner/SelectionQuerySegmentPruner.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9xdWVyeS9wcnVuZXIvU2VsZWN0aW9uUXVlcnlTZWdtZW50UHJ1bmVyLmphdmE=) | `82.95% <0.00%> (-3.41%)` | :arrow_down: |
   | [...ocal/indexsegment/immutable/EmptyIndexSegment.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9pbmRleHNlZ21lbnQvaW1tdXRhYmxlL0VtcHR5SW5kZXhTZWdtZW50LmphdmE=) | `0.00% <ø> (-36.85%)` | :arrow_down: |
   | [...ocal/indexsegment/mutable/IntermediateSegment.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9pbmRleHNlZ21lbnQvbXV0YWJsZS9JbnRlcm1lZGlhdGVTZWdtZW50LmphdmE=) | `0.00% <ø> (-70.07%)` | :arrow_down: |
   | [...local/realtime/impl/geospatial/MutableH3Index.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9yZWFsdGltZS9pbXBsL2dlb3NwYXRpYWwvTXV0YWJsZUgzSW5kZXguamF2YQ==) | `0.00% <ø> (-93.34%)` | :arrow_down: |
   | [...mpl/invertedindex/RealtimeInvertedIndexReader.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9yZWFsdGltZS9pbXBsL2ludmVydGVkaW5kZXgvUmVhbHRpbWVJbnZlcnRlZEluZGV4UmVhZGVyLmphdmE=) | `90.47% <ø> (-9.53%)` | :arrow_down: |
   | [...ealtime/impl/nullvalue/MutableNullValueVector.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9yZWFsdGltZS9pbXBsL251bGx2YWx1ZS9NdXRhYmxlTnVsbFZhbHVlVmVjdG9yLmphdmE=) | `85.71% <ø> (-14.29%)` | :arrow_down: |
   | [...segment/index/readers/ValidDocIndexReaderImpl.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9zZWdtZW50L2luZGV4L3JlYWRlcnMvVmFsaWREb2NJbmRleFJlYWRlckltcGwuamF2YQ==) | `0.00% <ø> (-100.00%)` | :arrow_down: |
   | [...not/segment/local/upsert/PartialUpsertHandler.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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==) | `0.00% <0.00%> (ø)` | |
   | [...t/segment/local/upsert/merger/IncrementMerger.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC91cHNlcnQvbWVyZ2VyL0luY3JlbWVudE1lcmdlci5qYXZh) | `0.00% <0.00%> (ø)` | |
   | [...t/segment/local/upsert/merger/OverwriteMerger.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC91cHNlcnQvbWVyZ2VyL092ZXJ3cml0ZU1lcmdlci5qYXZh) | `0.00% <0.00%> (ø)` | |
   | ... and [957 more](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6899?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/incubator-pinot/pull/6899?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 [282edba...c54b86f](https://codecov.io/gh/apache/incubator-pinot/pull/6899?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.

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] [incubator-pinot] codecov-commenter edited a comment on pull request #6899: Add partial upsert config and mergers

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #6899:
URL: https://github.com/apache/incubator-pinot/pull/6899#issuecomment-853568285






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

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



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


[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #6899: Add partial upsert config and mergers

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on a change in pull request #6899:
URL: https://github.com/apache/incubator-pinot/pull/6899#discussion_r655602084



##########
File path: pinot-core/src/test/java/org/apache/pinot/core/util/TableConfigUtilsTest.java
##########
@@ -1098,4 +1105,46 @@ public void testValidateUpsertConfig() {
       Assert.assertEquals(e.getMessage(), "The upsert table cannot have star-tree index.");
     }
   }
+
+  @Test
+  public void testValidatePartialUpsertConfig() {
+    Schema schema =
+        new Schema.SchemaBuilder().setSchemaName(TABLE_NAME).addSingleValueDimension("myCol1", FieldSpec.DataType.LONG)
+            .addSingleValueDimension("myCol2", FieldSpec.DataType.STRING)
+            .setPrimaryKeyColumns(Lists.newArrayList("myCol1")).build();
+
+    Map<String, String> streamConfigs = new HashMap<>();
+    streamConfigs.put("stream.kafka.consumer.type", "highLevel");
+    streamConfigs.put("streamType", "kafka");
+    streamConfigs.put("stream.kafka.topic.name", "test");
+    streamConfigs
+        .put("stream.kafka.decoder.class.name", "org.apache.pinot.plugin.stream.kafka.KafkaJSONMessageDecoder");
+    streamConfigs.put("stream.kafka.consumer.type", "simple");
+
+    TableConfig tableConfig = new TableConfigBuilder(TableType.REALTIME).setTableName(TABLE_NAME)
+        .setUpsertConfig(new UpsertConfig(UpsertConfig.Mode.PARTIAL)).setNullHandlingEnabled(false)
+        .setRoutingConfig(new RoutingConfig(null, null, RoutingConfig.STRICT_REPLICA_GROUP_INSTANCE_SELECTOR_TYPE))
+        .setStreamConfigs(streamConfigs).build();
+    try {
+      TableConfigUtils.validateUpsertConfig(tableConfig, schema);
+    } catch (Exception e) {
+      Assert.assertEquals(e.getMessage(), "NullValueHandling is required to be enabled for partial upsert tables.");
+    }
+
+    Map<String, UpsertConfig.Strategy> partialUpsertStratgies = new HashMap<>();
+    partialUpsertStratgies.put("myCol1", UpsertConfig.Strategy.INCREMENT);
+    try {
+      TableConfigUtils.validateUpsertConfig(tableConfig, schema);
+    } catch (Exception e) {
+      Assert.assertEquals(e.getMessage(), "INCREMENT merger cannot be applied to PK.");
+    }
+
+    partialUpsertStratgies = new HashMap<>();
+    partialUpsertStratgies.put("myCol2", UpsertConfig.Strategy.INCREMENT);
+    try {
+      TableConfigUtils.validateUpsertConfig(tableConfig, schema);
+    } catch (Exception e) {
+      Assert.assertEquals(e.getMessage(), "INCREMENT merger should be numeric data types.");

Review comment:
       Yes. IMO wait time should be long, instead of timestamp (milliseconds since epoch)




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

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



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


[GitHub] [incubator-pinot] codecov-commenter edited a comment on pull request #6899: Add partial upsert config and mergers

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #6899:
URL: https://github.com/apache/incubator-pinot/pull/6899#issuecomment-853568285


   # [Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6899?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 [#6899](https://codecov.io/gh/apache/incubator-pinot/pull/6899?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (13f8c83) into [master](https://codecov.io/gh/apache/incubator-pinot/commit/a1c9b631381a25ddd6d3164d6a9ce337c3939b9f?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (a1c9b63) will **decrease** coverage by `8.03%`.
   > The diff coverage is `33.64%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/incubator-pinot/pull/6899/graphs/tree.svg?width=650&height=150&src=pr&token=4ibza2ugkz&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/incubator-pinot/pull/6899?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #6899      +/-   ##
   ============================================
   - Coverage     73.38%   65.35%   -8.04%     
     Complexity       12       12              
   ============================================
     Files          1453     1458       +5     
     Lines         72032    72233     +201     
     Branches      10427    10469      +42     
   ============================================
   - Hits          52863    47208    -5655     
   - Misses        15643    21599    +5956     
   + Partials       3526     3426     -100     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration | `?` | |
   | unittests | `65.35% <33.64%> (-0.06%)` | :arrow_down: |
   
   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/incubator-pinot/pull/6899?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...ata/manager/realtime/RealtimeTableDataManager.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9kYXRhL21hbmFnZXIvcmVhbHRpbWUvUmVhbHRpbWVUYWJsZURhdGFNYW5hZ2VyLmphdmE=) | `10.65% <0.00%> (-58.13%)` | :arrow_down: |
   | [...ocal/indexsegment/immutable/EmptyIndexSegment.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9pbmRleHNlZ21lbnQvaW1tdXRhYmxlL0VtcHR5SW5kZXhTZWdtZW50LmphdmE=) | `36.84% <ø> (ø)` | |
   | [...ocal/indexsegment/mutable/IntermediateSegment.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9pbmRleHNlZ21lbnQvbXV0YWJsZS9JbnRlcm1lZGlhdGVTZWdtZW50LmphdmE=) | `70.06% <ø> (ø)` | |
   | [...local/realtime/impl/geospatial/MutableH3Index.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9yZWFsdGltZS9pbXBsL2dlb3NwYXRpYWwvTXV0YWJsZUgzSW5kZXguamF2YQ==) | `93.33% <ø> (ø)` | |
   | [...mpl/invertedindex/RealtimeInvertedIndexReader.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9yZWFsdGltZS9pbXBsL2ludmVydGVkaW5kZXgvUmVhbHRpbWVJbnZlcnRlZEluZGV4UmVhZGVyLmphdmE=) | `100.00% <ø> (ø)` | |
   | [...ealtime/impl/nullvalue/MutableNullValueVector.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9yZWFsdGltZS9pbXBsL251bGx2YWx1ZS9NdXRhYmxlTnVsbFZhbHVlVmVjdG9yLmphdmE=) | `85.71% <ø> (-14.29%)` | :arrow_down: |
   | [...segment/index/readers/ValidDocIndexReaderImpl.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9zZWdtZW50L2luZGV4L3JlYWRlcnMvVmFsaWREb2NJbmRleFJlYWRlckltcGwuamF2YQ==) | `0.00% <ø> (-100.00%)` | :arrow_down: |
   | [...not/segment/local/upsert/PartialUpsertHandler.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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==) | `0.00% <0.00%> (ø)` | |
   | [...t/segment/local/upsert/merger/IncrementMerger.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC91cHNlcnQvbWVyZ2VyL0luY3JlbWVudE1lcmdlci5qYXZh) | `0.00% <0.00%> (ø)` | |
   | [...t/segment/local/upsert/merger/OverwriteMerger.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC91cHNlcnQvbWVyZ2VyL092ZXJ3cml0ZU1lcmdlci5qYXZh) | `0.00% <0.00%> (ø)` | |
   | ... and [381 more](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6899?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/incubator-pinot/pull/6899?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 [a1c9b63...13f8c83](https://codecov.io/gh/apache/incubator-pinot/pull/6899?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.

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] [incubator-pinot] Jackie-Jiang commented on a change in pull request #6899: Add partial upsert config and mergers (WIP)

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on a change in pull request #6899:
URL: https://github.com/apache/incubator-pinot/pull/6899#discussion_r630453261



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/indexsegment/mutable/MutableSegmentImpl.java
##########
@@ -499,6 +515,29 @@ private boolean isUpsertEnabled() {
     return _upsertMode != UpsertConfig.Mode.NONE;
   }
 
+  private boolean isPartialUpsertEnabled() {
+    return _upsertMode == UpsertConfig.Mode.PARTIAL;
+  }
+
+  private GenericRow lookupAndMerge(GenericRow incomingRow, int docId) {
+    // get primary key and timestamp for the incoming record.
+    GenericRow previousRow = new GenericRow();
+    PrimaryKey primaryKey = incomingRow.getPrimaryKey(_schema.getPrimaryKeyColumns());
+    Object timeValue = incomingRow.getValue(_timeColumnName);
+    Preconditions.checkArgument(timeValue instanceof Comparable, "time column shall be comparable");
+    long timestamp = IngestionUtils.extractTimeValue((Comparable) timeValue);
+
+    // look up the previous full record with pk. Merge record if the incoming record is newer than previous record.
+    RecordLocation lastRecord = _partitionUpsertMetadataManager.findLastRecord(primaryKey);
+    if (timestamp >= lastRecord.getTimestamp()) {
+      previousRow = this.getRecord(lastRecord.getDocId(), previousRow);

Review comment:
       The record might not be in the current segment. You need to read the record from the segment pointed by the `RecordLocation`




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

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



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


[GitHub] [incubator-pinot] codecov-commenter edited a comment on pull request #6899: Add partial upsert config and mergers

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #6899:
URL: https://github.com/apache/incubator-pinot/pull/6899#issuecomment-853568285






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

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



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


[GitHub] [incubator-pinot] codecov-commenter commented on pull request #6899: Add partial upsert config and mergers (WIP)

Posted by GitBox <gi...@apache.org>.
codecov-commenter commented on pull request #6899:
URL: https://github.com/apache/incubator-pinot/pull/6899#issuecomment-853568285


   # [Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6899?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 [#6899](https://codecov.io/gh/apache/incubator-pinot/pull/6899?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (34b10b9) into [master](https://codecov.io/gh/apache/incubator-pinot/commit/0185482d9da2ac299b4b15bcd2998165ccbbdf71?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (0185482) will **decrease** coverage by `7.86%`.
   > The diff coverage is `32.00%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/incubator-pinot/pull/6899/graphs/tree.svg?width=650&height=150&src=pr&token=4ibza2ugkz&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/incubator-pinot/pull/6899?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #6899      +/-   ##
   ============================================
   - Coverage     73.23%   65.37%   -7.87%     
     Complexity       12       12              
   ============================================
     Files          1439     1455      +16     
     Lines         71333    71935     +602     
     Branches      10334    10433      +99     
   ============================================
   - Hits          52243    47029    -5214     
   - Misses        15578    21503    +5925     
   + Partials       3512     3403     -109     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration | `?` | |
   | unittests | `65.37% <32.00%> (+0.02%)` | :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://codecov.io/gh/apache/incubator-pinot/pull/6899?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...ata/manager/realtime/RealtimeTableDataManager.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9kYXRhL21hbmFnZXIvcmVhbHRpbWUvUmVhbHRpbWVUYWJsZURhdGFNYW5hZ2VyLmphdmE=) | `10.74% <0.00%> (-58.04%)` | :arrow_down: |
   | [...not/segment/local/upsert/PartialUpsertHandler.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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==) | `0.00% <0.00%> (ø)` | |
   | [...inot/segment/local/upsert/merger/IgnoreMerger.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC91cHNlcnQvbWVyZ2VyL0lnbm9yZU1lcmdlci5qYXZh) | `0.00% <0.00%> (ø)` | |
   | [...t/segment/local/upsert/merger/IncrementMerger.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC91cHNlcnQvbWVyZ2VyL0luY3JlbWVudE1lcmdlci5qYXZh) | `0.00% <0.00%> (ø)` | |
   | [...t/segment/local/upsert/merger/OverwriteMerger.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC91cHNlcnQvbWVyZ2VyL092ZXJ3cml0ZU1lcmdlci5qYXZh) | `0.00% <0.00%> (ø)` | |
   | [...he/pinot/segment/local/utils/TableConfigUtils.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC91dGlscy9UYWJsZUNvbmZpZ1V0aWxzLmphdmE=) | `75.20% <11.11%> (-6.67%)` | :arrow_down: |
   | [...t/local/upsert/PartitionUpsertMetadataManager.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC91cHNlcnQvUGFydGl0aW9uVXBzZXJ0TWV0YWRhdGFNYW5hZ2VyLmphdmE=) | `84.41% <38.88%> (-10.83%)` | :arrow_down: |
   | [...ent/local/realtime/impl/RealtimeSegmentConfig.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9yZWFsdGltZS9pbXBsL1JlYWx0aW1lU2VnbWVudENvbmZpZy5qYXZh) | `89.43% <55.55%> (-9.70%)` | :arrow_down: |
   | [...rg/apache/pinot/spi/config/table/UpsertConfig.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc3BpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcGkvY29uZmlnL3RhYmxlL1Vwc2VydENvbmZpZy5qYXZh) | `78.26% <73.33%> (-9.24%)` | :arrow_down: |
   | [...local/indexsegment/mutable/MutableSegmentImpl.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9pbmRleHNlZ21lbnQvbXV0YWJsZS9NdXRhYmxlU2VnbWVudEltcGwuamF2YQ==) | `66.21% <88.88%> (-12.65%)` | :arrow_down: |
   | ... and [384 more](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6899?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/incubator-pinot/pull/6899?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 [0185482...34b10b9](https://codecov.io/gh/apache/incubator-pinot/pull/6899?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.

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] [incubator-pinot] Jackie-Jiang merged pull request #6899: Add partial upsert config and mergers

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


   


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

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



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


[GitHub] [incubator-pinot] deemoliu commented on a change in pull request #6899: Add partial upsert config and mergers

Posted by GitBox <gi...@apache.org>.
deemoliu commented on a change in pull request #6899:
URL: https://github.com/apache/incubator-pinot/pull/6899#discussion_r655716908



##########
File path: pinot-tools/src/main/java/org/apache/pinot/tools/PartialUpsertQuickStart.java
##########
@@ -0,0 +1,113 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.tools;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import java.io.File;
+import java.net.URL;
+import org.apache.commons.io.FileUtils;
+import org.apache.pinot.common.utils.ZkStarter;
+import org.apache.pinot.spi.plugin.PluginManager;
+import org.apache.pinot.spi.stream.StreamDataProvider;
+import org.apache.pinot.spi.stream.StreamDataServerStartable;
+import org.apache.pinot.tools.Quickstart.Color;
+import org.apache.pinot.tools.admin.command.QuickstartRunner;
+import org.apache.pinot.tools.streams.MeetupRsvpStream;
+import org.apache.pinot.tools.utils.KafkaStarterUtils;
+
+import static org.apache.pinot.tools.Quickstart.prettyPrintResponse;
+import static org.apache.pinot.tools.Quickstart.printStatus;
+
+
+public class PartialUpsertQuickStart {
+  private StreamDataServerStartable _kafkaStarter;
+
+  public static void main(String[] args)
+      throws Exception {
+    PluginManager.get().init();
+    new PartialUpsertQuickStart().execute();
+  }
+
+  // Todo: add a quick start demo
+  public void execute()
+      throws Exception {
+    File quickstartTmpDir = new File(FileUtils.getTempDirectory(), String.valueOf(System.currentTimeMillis()));
+    File bootstrapTableDir = new File(quickstartTmpDir, "meetupRsvp");
+    File dataDir = new File(bootstrapTableDir, "data");
+    Preconditions.checkState(dataDir.mkdirs());
+
+    File schemaFile = new File(bootstrapTableDir, "meetupRsvp_schema.json");
+    File tableConfigFile = new File(bootstrapTableDir, "meetupRsvp_realtime_table_config.json");
+
+    ClassLoader classLoader = Quickstart.class.getClassLoader();
+    URL resource = classLoader.getResource("examples/stream/meetupRsvp/upsert_meetupRsvp_schema.json");
+    Preconditions.checkNotNull(resource);
+    FileUtils.copyURLToFile(resource, schemaFile);
+    resource =
+        classLoader.getResource("examples/stream/meetupRsvp/upsert_partial_meetupRsvp_realtime_table_config.json");
+    Preconditions.checkNotNull(resource);
+    FileUtils.copyURLToFile(resource, tableConfigFile);
+
+    QuickstartTableRequest request = new QuickstartTableRequest(bootstrapTableDir.getAbsolutePath());
+    final QuickstartRunner runner = new QuickstartRunner(Lists.newArrayList(request), 1, 1, 1, dataDir);
+
+    printStatus(Color.CYAN, "***** Starting Kafka *****");
+    final ZkStarter.ZookeeperInstance zookeeperInstance = ZkStarter.startLocalZkServer();
+    try {
+      _kafkaStarter = StreamDataProvider.getServerDataStartable(KafkaStarterUtils.KAFKA_SERVER_STARTABLE_CLASS_NAME,
+          KafkaStarterUtils.getDefaultKafkaConfiguration());
+    } catch (Exception e) {
+      throw new RuntimeException("Failed to start " + KafkaStarterUtils.KAFKA_SERVER_STARTABLE_CLASS_NAME, e);
+    }
+    _kafkaStarter.start();
+    _kafkaStarter.createTopic("meetupRSVPEvents", KafkaStarterUtils.getTopicCreationProps(2));
+    printStatus(Color.CYAN, "***** Starting  meetup data stream and publishing to Kafka *****");
+    MeetupRsvpStream meetupRSVPProvider = new MeetupRsvpStream(true);
+    meetupRSVPProvider.run();
+    printStatus(Color.CYAN, "***** Starting Zookeeper, controller, server and broker *****");
+    runner.startAll();
+    Runtime.getRuntime().addShutdownHook(new Thread(() -> {
+      try {
+        printStatus(Color.GREEN, "***** Shutting down realtime quick start *****");
+        runner.stop();
+        meetupRSVPProvider.stopPublishing();
+        _kafkaStarter.stop();
+        ZkStarter.stopLocalZkServer(zookeeperInstance);
+        FileUtils.deleteDirectory(quickstartTmpDir);
+      } catch (Exception e) {
+        e.printStackTrace();
+      }
+    }));
+    printStatus(Color.CYAN, "***** Bootstrap meetupRSVP(upsert) table *****");
+    runner.bootstrapTable();
+    printStatus(Color.CYAN, "***** Waiting for 15 seconds for a few events to get populated *****");
+    Thread.sleep(15000);
+
+    printStatus(Color.YELLOW, "***** Upsert quickstart setup complete *****");
+
+    String q1 = "select event_id, count(*), sum(rsvp_count) from meetupRsvp group by event_id order by sum(rsvp_count) desc limit 10";
+    printStatus(Color.YELLOW, "Total number of documents, total number of rsvp_counts per event_id in the table");

Review comment:
       gotcha, added comments.




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

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



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


[GitHub] [incubator-pinot] deemoliu commented on a change in pull request #6899: Add partial upsert config and mergers

Posted by GitBox <gi...@apache.org>.
deemoliu commented on a change in pull request #6899:
URL: https://github.com/apache/incubator-pinot/pull/6899#discussion_r655708998



##########
File path: pinot-segment-local/src/main/java/org/apache/pinot/segment/local/upsert/PartialUpsertHandler.java
##########
@@ -0,0 +1,141 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.segment.local.upsert;
+
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.helix.HelixDataAccessor;
+import org.apache.helix.HelixManager;
+import org.apache.helix.PropertyKey;
+import org.apache.helix.model.CurrentState;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.LiveInstance;
+import org.apache.pinot.segment.local.upsert.merger.PartialUpsertMerger;
+import org.apache.pinot.segment.local.upsert.merger.PartialUpsertMergerFactory;
+import org.apache.pinot.spi.config.table.UpsertConfig;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.utils.CommonConstants.Helix.StateModel.SegmentStateModel;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Handler for partial-upsert.
+ */
+public class PartialUpsertHandler {
+  private static final Logger LOGGER = LoggerFactory.getLogger(PartialUpsertHandler.class);
+
+  private final Map<String, PartialUpsertMerger> _mergers = new HashMap<>();

Review comment:
       gotcha, thanks for reviewing, will refactor this part.




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

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



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


[GitHub] [incubator-pinot] deemoliu commented on a change in pull request #6899: Add partial upsert config and mergers

Posted by GitBox <gi...@apache.org>.
deemoliu commented on a change in pull request #6899:
URL: https://github.com/apache/incubator-pinot/pull/6899#discussion_r656483680



##########
File path: pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java
##########
@@ -470,28 +466,32 @@ public void addExtraColumns(Schema newSchema) {
   @Override
   public boolean index(GenericRow row, @Nullable RowMetadata rowMetadata)
       throws IOException {
-    // Update dictionary first
-    updateDictionary(row);
-
-    // If metrics aggregation is enabled and if the dimension values were already seen, this will return existing docId,
-    // else this will return a new docId.
-    int docId = getOrCreateDocId();
-
     boolean canTakeMore;
-    if (docId == _numDocsIndexed) {
-      // New row
+    if (isUpsertEnabled()) {
+      row = handleUpsert(row, _numDocsIndexed);
+
+      updateDictionary(row);
       addNewRow(row);
       // Update number of documents indexed at last to make the latest row queryable
       canTakeMore = _numDocsIndexed++ < _capacity;
-
-      if (isUpsertEnabled()) {
-        handleUpsert(row, docId);
-      }
     } else {
-      Preconditions.checkArgument(!isUpsertEnabled(), "metrics aggregation cannot be used with upsert");
-      assert _aggregateMetrics;
-      aggregateMetrics(row, docId);
-      canTakeMore = true;
+      // Update dictionary first
+      updateDictionary(row);
+
+      // If metrics aggregation is enabled and if the dimension values were already seen, this will return existing
+      // docId, else this will return a new docId.
+      int docId = getOrCreateDocId();
+
+      if (docId == _numDocsIndexed) {
+        // New row
+        addNewRow(row);
+        // Update number of documents indexed at last to make the latest row queryable
+        canTakeMore = _numDocsIndexed++ < _capacity;
+      } else {
+        assert _aggregateMetrics;

Review comment:
       gotcha, let me remove 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.

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] [incubator-pinot] yupeng9 commented on a change in pull request #6899: Add partial upsert config and mergers (WIP)

Posted by GitBox <gi...@apache.org>.
yupeng9 commented on a change in pull request #6899:
URL: https://github.com/apache/incubator-pinot/pull/6899#discussion_r637704161



##########
File path: pinot-segment-local/src/main/java/org/apache/pinot/segment/local/upsert/PartialUpsertHandler.java
##########
@@ -0,0 +1,103 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.segment.local.upsert;
+
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.pinot.segment.local.upsert.merger.CustomMerger;
+import org.apache.pinot.segment.local.upsert.merger.IgnoreMerger;
+import org.apache.pinot.segment.local.upsert.merger.IncrementMerger;
+import org.apache.pinot.segment.local.upsert.merger.OverwriteMerger;
+import org.apache.pinot.segment.local.upsert.merger.PartialUpsertMerger;
+import org.apache.pinot.spi.config.table.UpsertConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+
+
+public class PartialUpsertHandler {
+  private HashMap<String, PartialUpsertMerger> _mergers;
+
+  /**
+   * Initializes the partial upsert merger with upsert config. Different fields can have different merge strategies.
+   *
+   * @param schema of table
+   * @param globalUpsertStrategy can be derived into fields to merger map.
+   * @param partialUpsertStrategy can be derived into fields to merger map.
+   * @param customMergeStrategy can be derived into fields to merger map.
+   */
+  public void init(Schema schema, UpsertConfig.STRATEGY globalUpsertStrategy,
+      Map<String, UpsertConfig.STRATEGY> partialUpsertStrategy, Map<String, String> customMergeStrategy) {
+    _mergers = new HashMap<>();
+
+    // init globalUpsertStrategy
+    if (globalUpsertStrategy == UpsertConfig.STRATEGY.IGNORE) {
+      for (String dimColumn : schema.getDimensionNames()) {
+        if (!schema.getPrimaryKeyColumns().contains(dimColumn)) {
+          _mergers.put(dimColumn, new IgnoreMerger(dimColumn));

Review comment:
       curious, why such merger initialization is needed? Cannot we use some static merger since they are inbuilt strategy?

##########
File path: pinot-segment-local/src/main/java/org/apache/pinot/segment/local/upsert/PartitionUpsertMetadataManager.java
##########
@@ -142,6 +148,33 @@ public ThreadSafeMutableRoaringBitmap addSegment(String segmentName, Iterator<Re
     return validDocIds;
   }
 
+  public RecordLocation findLastRecord(PrimaryKey primaryKey) {
+    RecordLocation currentRecordLocation = _primaryKeyToRecordLocationMap.get(primaryKey);
+    return currentRecordLocation;
+  }
+
+  public void handleUpsert(GenericRow row, int docId, PrimaryKey primaryKey, long timestamp,
+      MutableSegmentImpl mutableSegmentImpl) {
+    if (mutableSegmentImpl.isPartialUpsertEnabled()) {
+      // get primary key and timestamp for the incoming record.
+      GenericRow previousRow = new GenericRow();
+      // look up the previous full record with pk. Merge record if the incoming record is newer than previous record.
+      RecordLocation lastRecord = findLastRecord(primaryKey);

Review comment:
       can lastRecord be null, when it does not exist?

##########
File path: pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/TableConfigUtils.java
##########
@@ -348,6 +354,26 @@ public static void validateUpsertConfig(TableConfig tableConfig, Schema schema)
     Preconditions.checkState(
         CollectionUtils.isEmpty(tableConfig.getIndexingConfig().getStarTreeIndexConfigs()) && !tableConfig
             .getIndexingConfig().isEnableDefaultStarTree(), "The upsert table cannot have star-tree index.");
+
+    Preconditions.checkState(validatePartialUpsertStrategies(schema, tableConfig.getUpsertConfig()),
+        "The partial upsert strategies is not correct");
+  }
+
+  private static boolean validatePartialUpsertStrategies(Schema schema, UpsertConfig upsertConfig) {

Review comment:
       add javadoc on what this validates for

##########
File path: pinot-segment-local/src/main/java/org/apache/pinot/segment/local/upsert/merger/IgnoreMerger.java
##########
@@ -0,0 +1,38 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.segment.local.upsert.merger;
+
+import org.apache.pinot.spi.data.readers.GenericRow;
+
+
+public class IgnoreMerger implements PartialUpsertMerger {

Review comment:
       nit: We can create a static instance of this

##########
File path: pinot-segment-local/src/main/java/org/apache/pinot/segment/local/upsert/PartialUpsertHandler.java
##########
@@ -0,0 +1,103 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.segment.local.upsert;
+
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.pinot.segment.local.upsert.merger.CustomMerger;
+import org.apache.pinot.segment.local.upsert.merger.IgnoreMerger;
+import org.apache.pinot.segment.local.upsert.merger.IncrementMerger;
+import org.apache.pinot.segment.local.upsert.merger.OverwriteMerger;
+import org.apache.pinot.segment.local.upsert.merger.PartialUpsertMerger;
+import org.apache.pinot.spi.config.table.UpsertConfig;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.data.readers.GenericRow;
+
+
+public class PartialUpsertHandler {
+  private HashMap<String, PartialUpsertMerger> _mergers;
+
+  /**
+   * Initializes the partial upsert merger with upsert config. Different fields can have different merge strategies.
+   *
+   * @param schema of table
+   * @param globalUpsertStrategy can be derived into fields to merger map.
+   * @param partialUpsertStrategy can be derived into fields to merger map.
+   * @param customMergeStrategy can be derived into fields to merger map.
+   */
+  public void init(Schema schema, UpsertConfig.STRATEGY globalUpsertStrategy,
+      Map<String, UpsertConfig.STRATEGY> partialUpsertStrategy, Map<String, String> customMergeStrategy) {
+    _mergers = new HashMap<>();
+
+    // init globalUpsertStrategy
+    if (globalUpsertStrategy == UpsertConfig.STRATEGY.IGNORE) {
+      for (String dimColumn : schema.getDimensionNames()) {
+        if (!schema.getPrimaryKeyColumns().contains(dimColumn)) {
+          _mergers.put(dimColumn, new IgnoreMerger(dimColumn));
+        }
+      }
+      for (String metricColumn : schema.getMetricNames()) {
+        if (!schema.getPrimaryKeyColumns().contains(metricColumn)) {
+          _mergers.put(metricColumn, new IgnoreMerger(metricColumn));
+        }
+      }
+    } else {
+      for (String dimColumn : schema.getDimensionNames()) {

Review comment:
       no handling of INCREMENT?

##########
File path: pinot-spi/src/main/java/org/apache/pinot/spi/config/table/UpsertConfig.java
##########
@@ -30,16 +32,56 @@
     FULL, PARTIAL, NONE
   }
 
+  public enum STRATEGY {
+    OVERWRITE, IGNORE, INCREMENT
+  }
+
   private final Mode _mode;
+  private final STRATEGY _globalUpsertStrategy;
+  private final Map<String, STRATEGY> _partialUpsertStrategy;
+  private final Map<String, String> _customUpsertStrategy;
 
-  @JsonCreator
   public UpsertConfig(@JsonProperty(value = "mode", required = true) Mode mode) {
     Preconditions.checkArgument(mode != null, "Upsert mode must be configured");
     Preconditions.checkArgument(mode != Mode.PARTIAL, "Partial upsert mode is not supported");

Review comment:
       remove this?

##########
File path: pinot-segment-local/src/main/java/org/apache/pinot/segment/local/upsert/merger/PartialUpsertMerger.java
##########
@@ -0,0 +1,33 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.segment.local.upsert.merger;
+
+import org.apache.pinot.spi.data.readers.GenericRow;
+
+
+public interface PartialUpsertMerger {
+  /**
+   * Handle partial upsert merge for given fieldName.
+   *
+   * @param previousRecord the last derived full record during ingestion.
+   * @param currentRecord the new consumed record.
+   * @return a new row after merge
+   */
+  GenericRow merge(GenericRow previousRecord, GenericRow currentRecord);

Review comment:
       can previousRecord be null?

##########
File path: pinot-segment-local/src/main/java/org/apache/pinot/segment/local/upsert/merger/PartialUpsertMerger.java
##########
@@ -0,0 +1,33 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.segment.local.upsert.merger;
+
+import org.apache.pinot.spi.data.readers.GenericRow;
+
+
+public interface PartialUpsertMerger {

Review comment:
       javadoc




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

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



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


[GitHub] [incubator-pinot] yupeng9 commented on a change in pull request #6899: Add partial upsert config and mergers (WIP)

Posted by GitBox <gi...@apache.org>.
yupeng9 commented on a change in pull request #6899:
URL: https://github.com/apache/incubator-pinot/pull/6899#discussion_r637702155



##########
File path: pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java
##########
@@ -163,6 +164,8 @@
   private final ThreadSafeMutableRoaringBitmap _validDocIds;
   private final ValidDocIndexReader _validDocIndex;
 
+  private final PartialUpsertHandler _partialUpsertHandler = new PartialUpsertHandler();

Review comment:
       you can leave it to null, if partial upsert not enabled.




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

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



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


[GitHub] [incubator-pinot] yupeng9 commented on a change in pull request #6899: Add partial upsert config and mergers

Posted by GitBox <gi...@apache.org>.
yupeng9 commented on a change in pull request #6899:
URL: https://github.com/apache/incubator-pinot/pull/6899#discussion_r655555631



##########
File path: pinot-core/src/test/java/org/apache/pinot/core/util/TableConfigUtilsTest.java
##########
@@ -1098,4 +1105,46 @@ public void testValidateUpsertConfig() {
       Assert.assertEquals(e.getMessage(), "The upsert table cannot have star-tree index.");
     }
   }
+
+  @Test
+  public void testValidatePartialUpsertConfig() {
+    Schema schema =
+        new Schema.SchemaBuilder().setSchemaName(TABLE_NAME).addSingleValueDimension("myCol1", FieldSpec.DataType.LONG)
+            .addSingleValueDimension("myCol2", FieldSpec.DataType.STRING)
+            .setPrimaryKeyColumns(Lists.newArrayList("myCol1")).build();
+
+    Map<String, String> streamConfigs = new HashMap<>();
+    streamConfigs.put("stream.kafka.consumer.type", "highLevel");
+    streamConfigs.put("streamType", "kafka");
+    streamConfigs.put("stream.kafka.topic.name", "test");
+    streamConfigs
+        .put("stream.kafka.decoder.class.name", "org.apache.pinot.plugin.stream.kafka.KafkaJSONMessageDecoder");
+    streamConfigs.put("stream.kafka.consumer.type", "simple");
+
+    TableConfig tableConfig = new TableConfigBuilder(TableType.REALTIME).setTableName(TABLE_NAME)
+        .setUpsertConfig(new UpsertConfig(UpsertConfig.Mode.PARTIAL)).setNullHandlingEnabled(false)
+        .setRoutingConfig(new RoutingConfig(null, null, RoutingConfig.STRICT_REPLICA_GROUP_INSTANCE_SELECTOR_TYPE))
+        .setStreamConfigs(streamConfigs).build();
+    try {
+      TableConfigUtils.validateUpsertConfig(tableConfig, schema);
+    } catch (Exception e) {
+      Assert.assertEquals(e.getMessage(), "NullValueHandling is required to be enabled for partial upsert tables.");
+    }
+
+    Map<String, UpsertConfig.Strategy> partialUpsertStratgies = new HashMap<>();
+    partialUpsertStratgies.put("myCol1", UpsertConfig.Strategy.INCREMENT);
+    try {
+      TableConfigUtils.validateUpsertConfig(tableConfig, schema);
+    } catch (Exception e) {
+      Assert.assertEquals(e.getMessage(), "INCREMENT merger cannot be applied to PK.");
+    }
+
+    partialUpsertStratgies = new HashMap<>();
+    partialUpsertStratgies.put("myCol2", UpsertConfig.Strategy.INCREMENT);
+    try {
+      TableConfigUtils.validateUpsertConfig(tableConfig, schema);
+    } catch (Exception e) {
+      Assert.assertEquals(e.getMessage(), "INCREMENT merger should be numeric data types.");

Review comment:
       hmm, one example (contrived) is the driver/rider wait time, which can increment periodically (e.g. +5 min wait time)




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

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



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


[GitHub] [incubator-pinot] yupeng9 commented on a change in pull request #6899: Add partial upsert config and mergers (WIP)

Posted by GitBox <gi...@apache.org>.
yupeng9 commented on a change in pull request #6899:
URL: https://github.com/apache/incubator-pinot/pull/6899#discussion_r630532638



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/indexsegment/mutable/MutableSegmentImpl.java
##########
@@ -499,6 +515,29 @@ private boolean isUpsertEnabled() {
     return _upsertMode != UpsertConfig.Mode.NONE;
   }
 
+  private boolean isPartialUpsertEnabled() {
+    return _upsertMode == UpsertConfig.Mode.PARTIAL;
+  }
+
+  private GenericRow lookupAndMerge(GenericRow incomingRow, int docId) {
+    // get primary key and timestamp for the incoming record.
+    GenericRow previousRow = new GenericRow();
+    PrimaryKey primaryKey = incomingRow.getPrimaryKey(_schema.getPrimaryKeyColumns());
+    Object timeValue = incomingRow.getValue(_timeColumnName);
+    Preconditions.checkArgument(timeValue instanceof Comparable, "time column shall be comparable");
+    long timestamp = IngestionUtils.extractTimeValue((Comparable) timeValue);
+
+    // look up the previous full record with pk. Merge record if the incoming record is newer than previous record.
+    RecordLocation lastRecord = _partitionUpsertMetadataManager.findLastRecord(primaryKey);
+    if (timestamp >= lastRecord.getTimestamp()) {
+      previousRow = this.getRecord(lastRecord.getDocId(), previousRow);
+      return _partialUpsertHandler.merge(previousRow, incomingRow);

Review comment:
       this needs some error handling?

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/indexsegment/mutable/MutableSegmentImpl.java
##########
@@ -499,6 +515,29 @@ private boolean isUpsertEnabled() {
     return _upsertMode != UpsertConfig.Mode.NONE;
   }
 
+  private boolean isPartialUpsertEnabled() {
+    return _upsertMode == UpsertConfig.Mode.PARTIAL;
+  }
+
+  private GenericRow lookupAndMerge(GenericRow incomingRow, int docId) {
+    // get primary key and timestamp for the incoming record.
+    GenericRow previousRow = new GenericRow();
+    PrimaryKey primaryKey = incomingRow.getPrimaryKey(_schema.getPrimaryKeyColumns());
+    Object timeValue = incomingRow.getValue(_timeColumnName);
+    Preconditions.checkArgument(timeValue instanceof Comparable, "time column shall be comparable");
+    long timestamp = IngestionUtils.extractTimeValue((Comparable) timeValue);
+
+    // look up the previous full record with pk. Merge record if the incoming record is newer than previous record.
+    RecordLocation lastRecord = _partitionUpsertMetadataManager.findLastRecord(primaryKey);

Review comment:
       the last record might not exist?

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/realtime/impl/RealtimeSegmentConfig.java
##########
@@ -205,6 +214,18 @@ public PartitionUpsertMetadataManager getPartitionUpsertMetadataManager() {
     return _partitionUpsertMetadataManager;
   }
 
+  public UpsertConfig.STRATEGY getGlobalUpsertStrategy() {

Review comment:
       why do we need three strategies? I thought there is one strategy field, but can be of different types, such as predefined and custom in groovy

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/indexsegment/mutable/MutableSegmentImpl.java
##########
@@ -367,10 +372,16 @@ public long getLatestIngestionTimestamp() {
       _partitionUpsertMetadataManager = config.getPartitionUpsertMetadataManager();
       _validDocIds = new ThreadSafeMutableRoaringBitmap();
       _validDocIndex = new ValidDocIndexReaderImpl(_validDocIds);
+
+      // init partial upsert handler with partial upsert config

Review comment:
       we init this only for partial mode




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

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



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


[GitHub] [incubator-pinot] codecov-commenter edited a comment on pull request #6899: Add partial upsert config and mergers

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #6899:
URL: https://github.com/apache/incubator-pinot/pull/6899#issuecomment-853568285


   # [Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6899?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 [#6899](https://codecov.io/gh/apache/incubator-pinot/pull/6899?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (aa706ba) into [master](https://codecov.io/gh/apache/incubator-pinot/commit/01f206b1ef69ebaf27f0ac9fc1d92518b85bf0b1?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (01f206b) will **decrease** coverage by `7.89%`.
   > The diff coverage is `50.45%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/incubator-pinot/pull/6899/graphs/tree.svg?width=650&height=150&src=pr&token=4ibza2ugkz&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/incubator-pinot/pull/6899?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #6899      +/-   ##
   ============================================
   - Coverage     73.41%   65.51%   -7.90%     
   - Complexity       12       91      +79     
   ============================================
     Files          1454     1486      +32     
     Lines         72127    73071     +944     
     Branches      10448    10526      +78     
   ============================================
   - Hits          52951    47873    -5078     
   - Misses        15637    21818    +6181     
   + Partials       3539     3380     -159     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration | `?` | |
   | unittests | `65.51% <50.45%> (+0.09%)` | :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://codecov.io/gh/apache/incubator-pinot/pull/6899?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...ata/manager/realtime/RealtimeTableDataManager.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9kYXRhL21hbmFnZXIvcmVhbHRpbWUvUmVhbHRpbWVUYWJsZURhdGFNYW5hZ2VyLmphdmE=) | `10.69% <0.00%> (-58.08%)` | :arrow_down: |
   | [...ocal/indexsegment/immutable/EmptyIndexSegment.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9pbmRleHNlZ21lbnQvaW1tdXRhYmxlL0VtcHR5SW5kZXhTZWdtZW50LmphdmE=) | `36.84% <ø> (ø)` | |
   | [...ocal/indexsegment/mutable/IntermediateSegment.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9pbmRleHNlZ21lbnQvbXV0YWJsZS9JbnRlcm1lZGlhdGVTZWdtZW50LmphdmE=) | `70.06% <ø> (ø)` | |
   | [...local/realtime/impl/geospatial/MutableH3Index.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9yZWFsdGltZS9pbXBsL2dlb3NwYXRpYWwvTXV0YWJsZUgzSW5kZXguamF2YQ==) | `93.33% <ø> (ø)` | |
   | [...mpl/invertedindex/RealtimeInvertedIndexReader.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9yZWFsdGltZS9pbXBsL2ludmVydGVkaW5kZXgvUmVhbHRpbWVJbnZlcnRlZEluZGV4UmVhZGVyLmphdmE=) | `100.00% <ø> (ø)` | |
   | [...ealtime/impl/nullvalue/MutableNullValueVector.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9yZWFsdGltZS9pbXBsL251bGx2YWx1ZS9NdXRhYmxlTnVsbFZhbHVlVmVjdG9yLmphdmE=) | `85.71% <ø> (-14.29%)` | :arrow_down: |
   | [...segment/index/readers/ValidDocIndexReaderImpl.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9zZWdtZW50L2luZGV4L3JlYWRlcnMvVmFsaWREb2NJbmRleFJlYWRlckltcGwuamF2YQ==) | `0.00% <ø> (-100.00%)` | :arrow_down: |
   | [...ava/org/apache/pinot/segment/spi/IndexSegment.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1zcGkvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3Bpbm90L3NlZ21lbnQvc3BpL0luZGV4U2VnbWVudC5qYXZh) | `0.00% <ø> (ø)` | |
   | [...ment/spi/index/ThreadSafeMutableRoaringBitmap.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1zcGkvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3Bpbm90L3NlZ21lbnQvc3BpL2luZGV4L1RocmVhZFNhZmVNdXRhYmxlUm9hcmluZ0JpdG1hcC5qYXZh) | `100.00% <ø> (ø)` | |
   | [...ava/org/apache/pinot/core/plan/FilterPlanNode.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9wbGFuL0ZpbHRlclBsYW5Ob2RlLmphdmE=) | `72.91% <20.00%> (-6.25%)` | :arrow_down: |
   | ... and [625 more](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6899?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/incubator-pinot/pull/6899?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 [01f206b...aa706ba](https://codecov.io/gh/apache/incubator-pinot/pull/6899?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.

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] [incubator-pinot] yupeng9 commented on a change in pull request #6899: Add partial upsert config and mergers

Posted by GitBox <gi...@apache.org>.
yupeng9 commented on a change in pull request #6899:
URL: https://github.com/apache/incubator-pinot/pull/6899#discussion_r655590453



##########
File path: pinot-core/src/test/java/org/apache/pinot/core/util/TableConfigUtilsTest.java
##########
@@ -1098,4 +1105,46 @@ public void testValidateUpsertConfig() {
       Assert.assertEquals(e.getMessage(), "The upsert table cannot have star-tree index.");
     }
   }
+
+  @Test
+  public void testValidatePartialUpsertConfig() {
+    Schema schema =
+        new Schema.SchemaBuilder().setSchemaName(TABLE_NAME).addSingleValueDimension("myCol1", FieldSpec.DataType.LONG)
+            .addSingleValueDimension("myCol2", FieldSpec.DataType.STRING)
+            .setPrimaryKeyColumns(Lists.newArrayList("myCol1")).build();
+
+    Map<String, String> streamConfigs = new HashMap<>();
+    streamConfigs.put("stream.kafka.consumer.type", "highLevel");
+    streamConfigs.put("streamType", "kafka");
+    streamConfigs.put("stream.kafka.topic.name", "test");
+    streamConfigs
+        .put("stream.kafka.decoder.class.name", "org.apache.pinot.plugin.stream.kafka.KafkaJSONMessageDecoder");
+    streamConfigs.put("stream.kafka.consumer.type", "simple");
+
+    TableConfig tableConfig = new TableConfigBuilder(TableType.REALTIME).setTableName(TABLE_NAME)
+        .setUpsertConfig(new UpsertConfig(UpsertConfig.Mode.PARTIAL)).setNullHandlingEnabled(false)
+        .setRoutingConfig(new RoutingConfig(null, null, RoutingConfig.STRICT_REPLICA_GROUP_INSTANCE_SELECTOR_TYPE))
+        .setStreamConfigs(streamConfigs).build();
+    try {
+      TableConfigUtils.validateUpsertConfig(tableConfig, schema);
+    } catch (Exception e) {
+      Assert.assertEquals(e.getMessage(), "NullValueHandling is required to be enabled for partial upsert tables.");
+    }
+
+    Map<String, UpsertConfig.Strategy> partialUpsertStratgies = new HashMap<>();
+    partialUpsertStratgies.put("myCol1", UpsertConfig.Strategy.INCREMENT);
+    try {
+      TableConfigUtils.validateUpsertConfig(tableConfig, schema);
+    } catch (Exception e) {
+      Assert.assertEquals(e.getMessage(), "INCREMENT merger cannot be applied to PK.");
+    }
+
+    partialUpsertStratgies = new HashMap<>();
+    partialUpsertStratgies.put("myCol2", UpsertConfig.Strategy.INCREMENT);
+    try {
+      TableConfigUtils.validateUpsertConfig(tableConfig, schema);
+    } catch (Exception e) {
+      Assert.assertEquals(e.getMessage(), "INCREMENT merger should be numeric data types.");

Review comment:
       I see. By time column, I meant a dimension column of `Timestamp` type, which is added recently? Will the check fail this type?




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

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



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


[GitHub] [incubator-pinot] yupeng9 commented on a change in pull request #6899: Add partial upsert config and mergers

Posted by GitBox <gi...@apache.org>.
yupeng9 commented on a change in pull request #6899:
URL: https://github.com/apache/incubator-pinot/pull/6899#discussion_r650204656



##########
File path: pinot-segment-local/src/main/java/org/apache/pinot/segment/local/upsert/PartialUpsertHandler.java
##########
@@ -0,0 +1,138 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.segment.local.upsert;
+
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.helix.HelixDataAccessor;
+import org.apache.helix.HelixManager;
+import org.apache.helix.PropertyKey;
+import org.apache.helix.model.CurrentState;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.LiveInstance;
+import org.apache.pinot.segment.local.upsert.merger.PartialUpsertMerger;
+import org.apache.pinot.segment.local.upsert.merger.PartialUpsertMergerFactory;
+import org.apache.pinot.spi.config.table.UpsertConfig;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.utils.CommonConstants.Helix.StateModel.SegmentStateModel;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class PartialUpsertHandler {

Review comment:
       add javadoc

##########
File path: pinot-core/src/test/java/org/apache/pinot/core/util/TableConfigUtilsTest.java
##########
@@ -1098,4 +1105,46 @@ public void testValidateUpsertConfig() {
       Assert.assertEquals(e.getMessage(), "The upsert table cannot have star-tree index.");
     }
   }
+
+  @Test
+  public void testValidatePartialUpsertConfig() {
+    Schema schema =
+        new Schema.SchemaBuilder().setSchemaName(TABLE_NAME).addSingleValueDimension("myCol1", FieldSpec.DataType.LONG)
+            .addSingleValueDimension("myCol2", FieldSpec.DataType.STRING)
+            .setPrimaryKeyColumns(Lists.newArrayList("myCol1")).build();
+
+    Map<String, String> streamConfigs = new HashMap<>();
+    streamConfigs.put("stream.kafka.consumer.type", "highLevel");
+    streamConfigs.put("streamType", "kafka");
+    streamConfigs.put("stream.kafka.topic.name", "test");
+    streamConfigs
+        .put("stream.kafka.decoder.class.name", "org.apache.pinot.plugin.stream.kafka.KafkaJSONMessageDecoder");
+    streamConfigs.put("stream.kafka.consumer.type", "simple");
+
+    TableConfig tableConfig = new TableConfigBuilder(TableType.REALTIME).setTableName(TABLE_NAME)
+        .setUpsertConfig(new UpsertConfig(UpsertConfig.Mode.PARTIAL)).setNullHandlingEnabled(false)
+        .setRoutingConfig(new RoutingConfig(null, null, RoutingConfig.STRICT_REPLICA_GROUP_INSTANCE_SELECTOR_TYPE))
+        .setStreamConfigs(streamConfigs).build();
+    try {
+      TableConfigUtils.validateUpsertConfig(tableConfig, schema);
+    } catch (Exception e) {
+      Assert.assertEquals(e.getMessage(), "NullValueHandling is required to be enabled for partial upsert tables.");
+    }
+
+    Map<String, UpsertConfig.Strategy> partialUpsertStratgies = new HashMap<>();
+    partialUpsertStratgies.put("myCol1", UpsertConfig.Strategy.INCREMENT);
+    try {
+      TableConfigUtils.validateUpsertConfig(tableConfig, schema);
+    } catch (Exception e) {
+      Assert.assertEquals(e.getMessage(), "INCREMENT merger cannot be applied to PK.");
+    }
+
+    partialUpsertStratgies = new HashMap<>();
+    partialUpsertStratgies.put("myCol2", UpsertConfig.Strategy.INCREMENT);
+    try {
+      TableConfigUtils.validateUpsertConfig(tableConfig, schema);
+    } catch (Exception e) {
+      Assert.assertEquals(e.getMessage(), "INCREMENT merger should be numeric data types.");

Review comment:
       can increment merge be applied on time type?

##########
File path: pinot-segment-local/src/main/java/org/apache/pinot/segment/local/upsert/PartialUpsertHandler.java
##########
@@ -0,0 +1,138 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.segment.local.upsert;
+
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.helix.HelixDataAccessor;
+import org.apache.helix.HelixManager;
+import org.apache.helix.PropertyKey;
+import org.apache.helix.model.CurrentState;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.LiveInstance;
+import org.apache.pinot.segment.local.upsert.merger.PartialUpsertMerger;
+import org.apache.pinot.segment.local.upsert.merger.PartialUpsertMergerFactory;
+import org.apache.pinot.spi.config.table.UpsertConfig;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.utils.CommonConstants.Helix.StateModel.SegmentStateModel;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class PartialUpsertHandler {
+  private static final Logger LOGGER = LoggerFactory.getLogger(PartialUpsertHandler.class);
+
+  private final Map<String, PartialUpsertMerger> _mergers = new HashMap<>();
+
+  private final HelixManager _helixManager;
+  private final String _tableNameWithType;
+  private boolean _allSegmentsLoaded;

Review comment:
       this seems a cache, do we ever need to reset it?

##########
File path: pinot-spi/src/main/java/org/apache/pinot/spi/config/table/UpsertConfig.java
##########
@@ -30,16 +32,37 @@
     FULL, PARTIAL, NONE
   }
 
+  public enum Strategy {
+    OVERWRITE, INCREMENT

Review comment:
       add a TODO for custom

##########
File path: pinot-segment-local/src/main/java/org/apache/pinot/segment/local/upsert/merger/PartialUpsertMergerFactory.java
##########
@@ -0,0 +1,41 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.segment.local.upsert.merger;
+
+import org.apache.pinot.spi.config.table.UpsertConfig;
+
+
+public class PartialUpsertMergerFactory {
+  private PartialUpsertMergerFactory() {
+  }
+
+  private static final OverwriteMerger OVERWRITE_MERGER = new OverwriteMerger();
+  private static final IncrementMerger INCREMENT_MERGER = new IncrementMerger();
+
+  public static PartialUpsertMerger getMerger(UpsertConfig.Strategy strategy) {
+    switch (strategy) {
+      case OVERWRITE:
+        return OVERWRITE_MERGER;
+      case INCREMENT:
+        return INCREMENT_MERGER;

Review comment:
       do we have a test case to cover the case that the previous value does not exist, and the merger uses the increment value directly (instead of adding to the null-default value)?

##########
File path: pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/TableConfigUtils.java
##########
@@ -350,6 +357,36 @@ public static void validateUpsertConfig(TableConfig tableConfig, Schema schema)
             .getIndexingConfig().isEnableDefaultStarTree(), "The upsert table cannot have star-tree index.");
   }
 
+  /**
+   * Validates the partial upsert-related configurations
+   *  - INCREMENT merger cannot be applied to PK.
+   *  - INCREMENT merger should be numeric data types.
+   *  - enforce nullValueHandling for partial upsert tables.
+   */
+  private static void validatePartialUpsertStrategies(Schema schema, TableConfig tableConfig) {
+    if (tableConfig.getUpsertMode() != UpsertConfig.Mode.PARTIAL) {
+      return;
+    }
+
+    Preconditions.checkState(tableConfig.getIndexingConfig().isNullHandlingEnabled(),
+        "NullValueHandling is required to be enabled for partial upsert tables.");
+
+    Map<String, UpsertConfig.Strategy> partialUpsertStrategies =
+        tableConfig.getUpsertConfig().getPartialUpsertStrategies();
+
+    for (Map.Entry<String, UpsertConfig.Strategy> entry : partialUpsertStrategies.entrySet()) {
+      Set<FieldSpec.DataType> numericsDataType = new HashSet<>(Arrays.asList(INT, LONG, FLOAT, DOUBLE));

Review comment:
       shall allow datetime

##########
File path: pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java
##########
@@ -178,20 +175,20 @@ public MutableSegmentImpl(RealtimeSegmentConfig config, @Nullable ServerMetrics
             realtimeSegmentZKMetadata.getEndTime(), realtimeSegmentZKMetadata.getTimeUnit(),
             realtimeSegmentZKMetadata.getTotalDocs(), realtimeSegmentZKMetadata.getCrc(), _schema) {
           @Override
-      public int getTotalDocs() {
-        return _numDocsIndexed;
-      }
+          public int getTotalDocs() {
+            return _numDocsIndexed;
+          }
 
-      @Override
-      public long getLastIndexedTimestamp() {
-        return _lastIndexedTimeMs;
-      }
+          @Override
+          public long getLastIndexedTimestamp() {
+            return _lastIndexedTimeMs;
+          }
 
-      @Override
-      public long getLatestIngestionTimestamp() {
-        return _latestIngestionTimeMs;
-      }
-    };
+          @Override
+          public long getLatestIngestionTimestamp() {
+            return _latestIngestionTimeMs;

Review comment:
       why indentation change?

##########
File path: pinot-core/src/test/java/org/apache/pinot/core/util/TableConfigUtilsTest.java
##########
@@ -984,17 +989,19 @@ public void testValidateIndexingConfig() {
       // Expected
     }
 
-    tableConfig = new TableConfigBuilder(TableType.OFFLINE).setTableName(TABLE_NAME).setVarLengthDictionaryColumns(Arrays.asList("intCol")).
-        build();
+    tableConfig = new TableConfigBuilder(TableType.OFFLINE).setTableName(TABLE_NAME)
+        .setVarLengthDictionaryColumns(Arrays.asList("intCol")).

Review comment:
       curious, are you using a different formatter that resulted in these changes?

##########
File path: pinot-segment-local/src/main/java/org/apache/pinot/segment/local/upsert/PartitionUpsertMetadataManager.java
##########
@@ -60,128 +64,175 @@
   private final String _tableNameWithType;
   private final int _partitionId;
   private final ServerMetrics _serverMetrics;
+  private final PartialUpsertHandler _partialUpsertHandler;
 
-  public PartitionUpsertMetadataManager(String tableNameWithType, int partitionId, ServerMetrics serverMetrics) {
+  // TODO(upset): consider an off-heap KV store to persist this index to improve the recovery speed.
+  @VisibleForTesting
+  final ConcurrentHashMap<PrimaryKey, RecordLocation> _primaryKeyToRecordLocationMap = new ConcurrentHashMap<>();
+
+  // Reused for reading previous record during partial upsert
+  private final GenericRow _reuse = new GenericRow();
+  // Stores the result of updateRecord()
+  private GenericRow _result;
+
+  public PartitionUpsertMetadataManager(String tableNameWithType, int partitionId, ServerMetrics serverMetrics,
+      @Nullable PartialUpsertHandler partialUpsertHandler) {
     _tableNameWithType = tableNameWithType;
     _partitionId = partitionId;
     _serverMetrics = serverMetrics;
+    _partialUpsertHandler = partialUpsertHandler;
   }
 
-  public ConcurrentHashMap<PrimaryKey, RecordLocation> getPrimaryKeyToRecordLocationMap() {
-    return _primaryKeyToRecordLocationMap;
+  /**
+   * Initializes the upsert metadata for the given immutable segment.
+   */
+  public void addSegment(ImmutableSegmentImpl immutableSegment, Iterator<RecordInfo> recordInfoIterator) {
+    addSegment(immutableSegment, recordInfoIterator, new ThreadSafeMutableRoaringBitmap());
   }
 
-  // TODO(upset): consider an off-heap KV store to persist this index to improve the recovery speed.
   @VisibleForTesting
-  final ConcurrentHashMap<PrimaryKey, RecordLocation> _primaryKeyToRecordLocationMap = new ConcurrentHashMap<>();
-
-  /**
-   * Initializes the upsert metadata for the given immutable segment, returns the valid doc ids for the segment.
-   */
-  public ThreadSafeMutableRoaringBitmap addSegment(String segmentName, Iterator<RecordInfo> recordInfoIterator) {
+  void addSegment(ImmutableSegmentImpl immutableSegment, Iterator<RecordInfo> recordInfoIterator,
+      ThreadSafeMutableRoaringBitmap validDocIds) {
+    String segmentName = immutableSegment.getSegmentName();
     LOGGER.info("Adding upsert metadata for segment: {}", segmentName);
+    immutableSegment.enableUpsert(this, validDocIds);
 
-    ThreadSafeMutableRoaringBitmap validDocIds = new ThreadSafeMutableRoaringBitmap();
     while (recordInfoIterator.hasNext()) {
       RecordInfo recordInfo = recordInfoIterator.next();
       _primaryKeyToRecordLocationMap.compute(recordInfo._primaryKey, (primaryKey, currentRecordLocation) -> {
         if (currentRecordLocation != null) {
           // Existing primary key
 
-          if (segmentName.equals(currentRecordLocation.getSegmentName())) {
-            // The current record location has the same segment name
-
-            // Update the record location when the new timestamp is greater than or equal to the current timestamp.
-            // There are 2 scenarios:
-            //   1. The current record location is pointing to the same segment (the segment being added). In this case,
-            //      we want to update the record location when there is a tie to keep the newer record. Note that the
-            //      record info iterator will return records with incremental doc ids.
-            //   2. The current record location is pointing to the old segment being replaced. This could happen when
-            //      committing a consuming segment, or reloading a completed segment. In this case, we want to update
-            //      the record location when there is a tie because the record locations should point to the new added
-            //      segment instead of the old segment being replaced. Also, do not update the valid doc ids for the old
-            //      segment because it has not been replaced yet.
+          // The current record is in the same segment
+          // Update the record location when there is a tie to keep the newer record. Note that the record info iterator
+          // will return records with incremental doc ids.
+          IndexSegment currentSegment = currentRecordLocation.getSegment();
+          if (immutableSegment == currentSegment) {
             if (recordInfo._timestamp >= currentRecordLocation.getTimestamp()) {
-              // Only update the valid doc ids for the new segment
-              if (validDocIds == currentRecordLocation.getValidDocIds()) {
-                validDocIds.remove(currentRecordLocation.getDocId());
-              }
+              validDocIds.remove(currentRecordLocation.getDocId());
               validDocIds.add(recordInfo._docId);
-              return new RecordLocation(segmentName, recordInfo._docId, recordInfo._timestamp, validDocIds);
+              return new RecordLocation(immutableSegment, recordInfo._docId, recordInfo._timestamp);
             } else {
               return currentRecordLocation;
             }
-          } else {
-            // The current record location is pointing to a different segment
-
-            // Update the record location when getting a newer timestamp, or the timestamp is the same as the current
-            // timestamp, but the segment has a larger sequence number (the segment is newer than the current segment).
-            if (recordInfo._timestamp > currentRecordLocation.getTimestamp() || (
-                recordInfo._timestamp == currentRecordLocation.getTimestamp()
-                    && LLCSegmentName.isLowLevelConsumerSegmentName(segmentName)
-                    && LLCSegmentName.isLowLevelConsumerSegmentName(currentRecordLocation.getSegmentName())
-                    && LLCSegmentName.getSequenceNumber(segmentName) > LLCSegmentName
-                    .getSequenceNumber(currentRecordLocation.getSegmentName()))) {
-              currentRecordLocation.getValidDocIds().remove(currentRecordLocation.getDocId());
+          }
+
+          // The current record is in an old segment being replaced
+          // This could happen when committing a consuming segment, or reloading a completed segment. In this case, we
+          // want to update the record location when there is a tie because the record locations should point to the new
+          // added segment instead of the old segment being replaced. Also, do not update the valid doc ids for the old
+          // segment because it has not been replaced yet.
+          String currentSegmentName = currentSegment.getSegmentName();
+          if (segmentName.equals(currentSegmentName)) {
+            if (recordInfo._timestamp >= currentRecordLocation.getTimestamp()) {
               validDocIds.add(recordInfo._docId);
-              return new RecordLocation(segmentName, recordInfo._docId, recordInfo._timestamp, validDocIds);
+              return new RecordLocation(immutableSegment, recordInfo._docId, recordInfo._timestamp);
             } else {
               return currentRecordLocation;
             }
           }
+
+          // The current record is in a different segment
+          // Update the record location when getting a newer timestamp, or the timestamp is the same as the current
+          // timestamp, but the segment has a larger sequence number (the segment is newer than the current segment).
+          if (recordInfo._timestamp > currentRecordLocation.getTimestamp() || (
+              recordInfo._timestamp == currentRecordLocation.getTimestamp() && LLCSegmentName
+                  .isLowLevelConsumerSegmentName(segmentName) && LLCSegmentName
+                  .isLowLevelConsumerSegmentName(currentSegmentName)
+                  && LLCSegmentName.getSequenceNumber(segmentName) > LLCSegmentName
+                  .getSequenceNumber(currentSegmentName))) {
+            assert currentSegment.getValidDocIds() != null;
+            currentSegment.getValidDocIds().remove(currentRecordLocation.getDocId());
+            validDocIds.add(recordInfo._docId);
+            return new RecordLocation(immutableSegment, recordInfo._docId, recordInfo._timestamp);
+          } else {
+            return currentRecordLocation;
+          }
         } else {
           // New primary key
           validDocIds.add(recordInfo._docId);
-          return new RecordLocation(segmentName, recordInfo._docId, recordInfo._timestamp, validDocIds);
+          return new RecordLocation(immutableSegment, recordInfo._docId, recordInfo._timestamp);
         }
       });
     }
     // Update metrics
     _serverMetrics.setValueOfPartitionGauge(_tableNameWithType, _partitionId, ServerGauge.UPSERT_PRIMARY_KEYS_COUNT,
         _primaryKeyToRecordLocationMap.size());
-    return validDocIds;
   }
 
   /**
-   * Updates the upsert metadata for a new consumed record in the given consuming segment.
+   * Updates the upsert metadata for a new consumed record in the given consuming segment. Returns the merged record if
+   * partial-upsert is enabled.
    */
-  public void updateRecord(String segmentName, RecordInfo recordInfo, ThreadSafeMutableRoaringBitmap validDocIds) {
+  public GenericRow updateRecord(IndexSegment segment, RecordInfo recordInfo, GenericRow record) {
+    // For partial-upsert, need to ensure all previous records are loaded before inserting new records.
+    if (_partialUpsertHandler != null) {
+      while (!_partialUpsertHandler.isAllSegmentsLoaded()) {
+        LOGGER
+            .info("Sleeping 1 second waiting for all segments loaded for partial-upsert table: {}", _tableNameWithType);
+        try {
+          //noinspection BusyWait
+          Thread.sleep(1000L);
+        } catch (InterruptedException e) {
+          throw new RuntimeException(e);
+        }
+      }
+    }
+
+    _result = record;
     _primaryKeyToRecordLocationMap.compute(recordInfo._primaryKey, (primaryKey, currentRecordLocation) -> {
       if (currentRecordLocation != null) {
         // Existing primary key
 
         // Update the record location when the new timestamp is greater than or equal to the current timestamp. Update
         // the record location when there is a tie to keep the newer record.
         if (recordInfo._timestamp >= currentRecordLocation.getTimestamp()) {
-          currentRecordLocation.getValidDocIds().remove(currentRecordLocation.getDocId());
-          validDocIds.add(recordInfo._docId);
-          return new RecordLocation(segmentName, recordInfo._docId, recordInfo._timestamp, validDocIds);
+          IndexSegment currentSegment = currentRecordLocation.getSegment();
+          if (_partialUpsertHandler != null) {
+            // Partial upsert
+            GenericRow previousRecord = currentSegment.getRecord(currentRecordLocation.getDocId(), _reuse);

Review comment:
       can `previousRecord ` be null if not found?

##########
File path: pinot-segment-local/src/main/java/org/apache/pinot/segment/local/upsert/PartitionUpsertMetadataManager.java
##########
@@ -60,128 +64,175 @@
   private final String _tableNameWithType;
   private final int _partitionId;
   private final ServerMetrics _serverMetrics;
+  private final PartialUpsertHandler _partialUpsertHandler;
 
-  public PartitionUpsertMetadataManager(String tableNameWithType, int partitionId, ServerMetrics serverMetrics) {
+  // TODO(upset): consider an off-heap KV store to persist this index to improve the recovery speed.
+  @VisibleForTesting
+  final ConcurrentHashMap<PrimaryKey, RecordLocation> _primaryKeyToRecordLocationMap = new ConcurrentHashMap<>();
+
+  // Reused for reading previous record during partial upsert
+  private final GenericRow _reuse = new GenericRow();
+  // Stores the result of updateRecord()
+  private GenericRow _result;
+
+  public PartitionUpsertMetadataManager(String tableNameWithType, int partitionId, ServerMetrics serverMetrics,
+      @Nullable PartialUpsertHandler partialUpsertHandler) {
     _tableNameWithType = tableNameWithType;
     _partitionId = partitionId;
     _serverMetrics = serverMetrics;
+    _partialUpsertHandler = partialUpsertHandler;
   }
 
-  public ConcurrentHashMap<PrimaryKey, RecordLocation> getPrimaryKeyToRecordLocationMap() {
-    return _primaryKeyToRecordLocationMap;
+  /**
+   * Initializes the upsert metadata for the given immutable segment.
+   */
+  public void addSegment(ImmutableSegmentImpl immutableSegment, Iterator<RecordInfo> recordInfoIterator) {
+    addSegment(immutableSegment, recordInfoIterator, new ThreadSafeMutableRoaringBitmap());
   }
 
-  // TODO(upset): consider an off-heap KV store to persist this index to improve the recovery speed.
   @VisibleForTesting
-  final ConcurrentHashMap<PrimaryKey, RecordLocation> _primaryKeyToRecordLocationMap = new ConcurrentHashMap<>();
-
-  /**
-   * Initializes the upsert metadata for the given immutable segment, returns the valid doc ids for the segment.
-   */
-  public ThreadSafeMutableRoaringBitmap addSegment(String segmentName, Iterator<RecordInfo> recordInfoIterator) {
+  void addSegment(ImmutableSegmentImpl immutableSegment, Iterator<RecordInfo> recordInfoIterator,
+      ThreadSafeMutableRoaringBitmap validDocIds) {
+    String segmentName = immutableSegment.getSegmentName();
     LOGGER.info("Adding upsert metadata for segment: {}", segmentName);
+    immutableSegment.enableUpsert(this, validDocIds);
 
-    ThreadSafeMutableRoaringBitmap validDocIds = new ThreadSafeMutableRoaringBitmap();
     while (recordInfoIterator.hasNext()) {
       RecordInfo recordInfo = recordInfoIterator.next();
       _primaryKeyToRecordLocationMap.compute(recordInfo._primaryKey, (primaryKey, currentRecordLocation) -> {
         if (currentRecordLocation != null) {
           // Existing primary key
 
-          if (segmentName.equals(currentRecordLocation.getSegmentName())) {
-            // The current record location has the same segment name
-
-            // Update the record location when the new timestamp is greater than or equal to the current timestamp.
-            // There are 2 scenarios:
-            //   1. The current record location is pointing to the same segment (the segment being added). In this case,
-            //      we want to update the record location when there is a tie to keep the newer record. Note that the
-            //      record info iterator will return records with incremental doc ids.
-            //   2. The current record location is pointing to the old segment being replaced. This could happen when
-            //      committing a consuming segment, or reloading a completed segment. In this case, we want to update
-            //      the record location when there is a tie because the record locations should point to the new added
-            //      segment instead of the old segment being replaced. Also, do not update the valid doc ids for the old
-            //      segment because it has not been replaced yet.
+          // The current record is in the same segment
+          // Update the record location when there is a tie to keep the newer record. Note that the record info iterator
+          // will return records with incremental doc ids.
+          IndexSegment currentSegment = currentRecordLocation.getSegment();
+          if (immutableSegment == currentSegment) {
             if (recordInfo._timestamp >= currentRecordLocation.getTimestamp()) {
-              // Only update the valid doc ids for the new segment
-              if (validDocIds == currentRecordLocation.getValidDocIds()) {
-                validDocIds.remove(currentRecordLocation.getDocId());
-              }
+              validDocIds.remove(currentRecordLocation.getDocId());
               validDocIds.add(recordInfo._docId);
-              return new RecordLocation(segmentName, recordInfo._docId, recordInfo._timestamp, validDocIds);
+              return new RecordLocation(immutableSegment, recordInfo._docId, recordInfo._timestamp);
             } else {
               return currentRecordLocation;
             }
-          } else {
-            // The current record location is pointing to a different segment
-
-            // Update the record location when getting a newer timestamp, or the timestamp is the same as the current
-            // timestamp, but the segment has a larger sequence number (the segment is newer than the current segment).
-            if (recordInfo._timestamp > currentRecordLocation.getTimestamp() || (
-                recordInfo._timestamp == currentRecordLocation.getTimestamp()
-                    && LLCSegmentName.isLowLevelConsumerSegmentName(segmentName)
-                    && LLCSegmentName.isLowLevelConsumerSegmentName(currentRecordLocation.getSegmentName())
-                    && LLCSegmentName.getSequenceNumber(segmentName) > LLCSegmentName
-                    .getSequenceNumber(currentRecordLocation.getSegmentName()))) {
-              currentRecordLocation.getValidDocIds().remove(currentRecordLocation.getDocId());
+          }
+
+          // The current record is in an old segment being replaced
+          // This could happen when committing a consuming segment, or reloading a completed segment. In this case, we
+          // want to update the record location when there is a tie because the record locations should point to the new
+          // added segment instead of the old segment being replaced. Also, do not update the valid doc ids for the old
+          // segment because it has not been replaced yet.
+          String currentSegmentName = currentSegment.getSegmentName();
+          if (segmentName.equals(currentSegmentName)) {
+            if (recordInfo._timestamp >= currentRecordLocation.getTimestamp()) {
               validDocIds.add(recordInfo._docId);
-              return new RecordLocation(segmentName, recordInfo._docId, recordInfo._timestamp, validDocIds);
+              return new RecordLocation(immutableSegment, recordInfo._docId, recordInfo._timestamp);
             } else {
               return currentRecordLocation;
             }
           }
+
+          // The current record is in a different segment
+          // Update the record location when getting a newer timestamp, or the timestamp is the same as the current
+          // timestamp, but the segment has a larger sequence number (the segment is newer than the current segment).
+          if (recordInfo._timestamp > currentRecordLocation.getTimestamp() || (
+              recordInfo._timestamp == currentRecordLocation.getTimestamp() && LLCSegmentName
+                  .isLowLevelConsumerSegmentName(segmentName) && LLCSegmentName
+                  .isLowLevelConsumerSegmentName(currentSegmentName)
+                  && LLCSegmentName.getSequenceNumber(segmentName) > LLCSegmentName
+                  .getSequenceNumber(currentSegmentName))) {
+            assert currentSegment.getValidDocIds() != null;
+            currentSegment.getValidDocIds().remove(currentRecordLocation.getDocId());
+            validDocIds.add(recordInfo._docId);
+            return new RecordLocation(immutableSegment, recordInfo._docId, recordInfo._timestamp);
+          } else {
+            return currentRecordLocation;
+          }
         } else {
           // New primary key
           validDocIds.add(recordInfo._docId);
-          return new RecordLocation(segmentName, recordInfo._docId, recordInfo._timestamp, validDocIds);
+          return new RecordLocation(immutableSegment, recordInfo._docId, recordInfo._timestamp);
         }
       });
     }
     // Update metrics
     _serverMetrics.setValueOfPartitionGauge(_tableNameWithType, _partitionId, ServerGauge.UPSERT_PRIMARY_KEYS_COUNT,
         _primaryKeyToRecordLocationMap.size());
-    return validDocIds;
   }
 
   /**
-   * Updates the upsert metadata for a new consumed record in the given consuming segment.
+   * Updates the upsert metadata for a new consumed record in the given consuming segment. Returns the merged record if
+   * partial-upsert is enabled.
    */
-  public void updateRecord(String segmentName, RecordInfo recordInfo, ThreadSafeMutableRoaringBitmap validDocIds) {
+  public GenericRow updateRecord(IndexSegment segment, RecordInfo recordInfo, GenericRow record) {
+    // For partial-upsert, need to ensure all previous records are loaded before inserting new records.
+    if (_partialUpsertHandler != null) {
+      while (!_partialUpsertHandler.isAllSegmentsLoaded()) {
+        LOGGER
+            .info("Sleeping 1 second waiting for all segments loaded for partial-upsert table: {}", _tableNameWithType);

Review comment:
       is it possible that segment seals during this 1s?

##########
File path: pinot-segment-local/src/main/java/org/apache/pinot/segment/local/upsert/PartitionUpsertMetadataManager.java
##########
@@ -60,128 +64,175 @@
   private final String _tableNameWithType;
   private final int _partitionId;
   private final ServerMetrics _serverMetrics;
+  private final PartialUpsertHandler _partialUpsertHandler;
 
-  public PartitionUpsertMetadataManager(String tableNameWithType, int partitionId, ServerMetrics serverMetrics) {
+  // TODO(upset): consider an off-heap KV store to persist this index to improve the recovery speed.
+  @VisibleForTesting
+  final ConcurrentHashMap<PrimaryKey, RecordLocation> _primaryKeyToRecordLocationMap = new ConcurrentHashMap<>();
+
+  // Reused for reading previous record during partial upsert
+  private final GenericRow _reuse = new GenericRow();
+  // Stores the result of updateRecord()
+  private GenericRow _result;
+
+  public PartitionUpsertMetadataManager(String tableNameWithType, int partitionId, ServerMetrics serverMetrics,
+      @Nullable PartialUpsertHandler partialUpsertHandler) {
     _tableNameWithType = tableNameWithType;
     _partitionId = partitionId;
     _serverMetrics = serverMetrics;
+    _partialUpsertHandler = partialUpsertHandler;
   }
 
-  public ConcurrentHashMap<PrimaryKey, RecordLocation> getPrimaryKeyToRecordLocationMap() {
-    return _primaryKeyToRecordLocationMap;
+  /**
+   * Initializes the upsert metadata for the given immutable segment.
+   */
+  public void addSegment(ImmutableSegmentImpl immutableSegment, Iterator<RecordInfo> recordInfoIterator) {
+    addSegment(immutableSegment, recordInfoIterator, new ThreadSafeMutableRoaringBitmap());
   }
 
-  // TODO(upset): consider an off-heap KV store to persist this index to improve the recovery speed.
   @VisibleForTesting
-  final ConcurrentHashMap<PrimaryKey, RecordLocation> _primaryKeyToRecordLocationMap = new ConcurrentHashMap<>();
-
-  /**
-   * Initializes the upsert metadata for the given immutable segment, returns the valid doc ids for the segment.
-   */
-  public ThreadSafeMutableRoaringBitmap addSegment(String segmentName, Iterator<RecordInfo> recordInfoIterator) {
+  void addSegment(ImmutableSegmentImpl immutableSegment, Iterator<RecordInfo> recordInfoIterator,
+      ThreadSafeMutableRoaringBitmap validDocIds) {
+    String segmentName = immutableSegment.getSegmentName();
     LOGGER.info("Adding upsert metadata for segment: {}", segmentName);
+    immutableSegment.enableUpsert(this, validDocIds);
 
-    ThreadSafeMutableRoaringBitmap validDocIds = new ThreadSafeMutableRoaringBitmap();
     while (recordInfoIterator.hasNext()) {
       RecordInfo recordInfo = recordInfoIterator.next();
       _primaryKeyToRecordLocationMap.compute(recordInfo._primaryKey, (primaryKey, currentRecordLocation) -> {
         if (currentRecordLocation != null) {
           // Existing primary key
 
-          if (segmentName.equals(currentRecordLocation.getSegmentName())) {
-            // The current record location has the same segment name
-
-            // Update the record location when the new timestamp is greater than or equal to the current timestamp.
-            // There are 2 scenarios:
-            //   1. The current record location is pointing to the same segment (the segment being added). In this case,
-            //      we want to update the record location when there is a tie to keep the newer record. Note that the
-            //      record info iterator will return records with incremental doc ids.
-            //   2. The current record location is pointing to the old segment being replaced. This could happen when
-            //      committing a consuming segment, or reloading a completed segment. In this case, we want to update
-            //      the record location when there is a tie because the record locations should point to the new added
-            //      segment instead of the old segment being replaced. Also, do not update the valid doc ids for the old
-            //      segment because it has not been replaced yet.
+          // The current record is in the same segment
+          // Update the record location when there is a tie to keep the newer record. Note that the record info iterator
+          // will return records with incremental doc ids.
+          IndexSegment currentSegment = currentRecordLocation.getSegment();
+          if (immutableSegment == currentSegment) {
             if (recordInfo._timestamp >= currentRecordLocation.getTimestamp()) {
-              // Only update the valid doc ids for the new segment
-              if (validDocIds == currentRecordLocation.getValidDocIds()) {
-                validDocIds.remove(currentRecordLocation.getDocId());
-              }
+              validDocIds.remove(currentRecordLocation.getDocId());
               validDocIds.add(recordInfo._docId);
-              return new RecordLocation(segmentName, recordInfo._docId, recordInfo._timestamp, validDocIds);
+              return new RecordLocation(immutableSegment, recordInfo._docId, recordInfo._timestamp);
             } else {
               return currentRecordLocation;
             }
-          } else {
-            // The current record location is pointing to a different segment
-
-            // Update the record location when getting a newer timestamp, or the timestamp is the same as the current
-            // timestamp, but the segment has a larger sequence number (the segment is newer than the current segment).
-            if (recordInfo._timestamp > currentRecordLocation.getTimestamp() || (
-                recordInfo._timestamp == currentRecordLocation.getTimestamp()
-                    && LLCSegmentName.isLowLevelConsumerSegmentName(segmentName)
-                    && LLCSegmentName.isLowLevelConsumerSegmentName(currentRecordLocation.getSegmentName())
-                    && LLCSegmentName.getSequenceNumber(segmentName) > LLCSegmentName
-                    .getSequenceNumber(currentRecordLocation.getSegmentName()))) {
-              currentRecordLocation.getValidDocIds().remove(currentRecordLocation.getDocId());
+          }
+
+          // The current record is in an old segment being replaced
+          // This could happen when committing a consuming segment, or reloading a completed segment. In this case, we
+          // want to update the record location when there is a tie because the record locations should point to the new
+          // added segment instead of the old segment being replaced. Also, do not update the valid doc ids for the old
+          // segment because it has not been replaced yet.
+          String currentSegmentName = currentSegment.getSegmentName();
+          if (segmentName.equals(currentSegmentName)) {
+            if (recordInfo._timestamp >= currentRecordLocation.getTimestamp()) {
               validDocIds.add(recordInfo._docId);
-              return new RecordLocation(segmentName, recordInfo._docId, recordInfo._timestamp, validDocIds);
+              return new RecordLocation(immutableSegment, recordInfo._docId, recordInfo._timestamp);
             } else {
               return currentRecordLocation;
             }
           }
+
+          // The current record is in a different segment
+          // Update the record location when getting a newer timestamp, or the timestamp is the same as the current
+          // timestamp, but the segment has a larger sequence number (the segment is newer than the current segment).

Review comment:
       why do we need this additional handling and split the handling of immutable segment vs consuming segment? Why does partial upsert require this change?

##########
File path: pinot-segment-local/src/main/java/org/apache/pinot/segment/local/upsert/merger/IncrementMerger.java
##########
@@ -0,0 +1,44 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.segment.local.upsert.merger;
+
+public class IncrementMerger implements PartialUpsertMerger {
+  IncrementMerger() {

Review comment:
       private

##########
File path: pinot-segment-local/src/main/java/org/apache/pinot/segment/local/upsert/PartialUpsertHandler.java
##########
@@ -0,0 +1,138 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.segment.local.upsert;
+
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.helix.HelixDataAccessor;
+import org.apache.helix.HelixManager;
+import org.apache.helix.PropertyKey;
+import org.apache.helix.model.CurrentState;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.LiveInstance;
+import org.apache.pinot.segment.local.upsert.merger.PartialUpsertMerger;
+import org.apache.pinot.segment.local.upsert.merger.PartialUpsertMergerFactory;
+import org.apache.pinot.spi.config.table.UpsertConfig;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.utils.CommonConstants.Helix.StateModel.SegmentStateModel;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class PartialUpsertHandler {
+  private static final Logger LOGGER = LoggerFactory.getLogger(PartialUpsertHandler.class);
+
+  private final Map<String, PartialUpsertMerger> _mergers = new HashMap<>();
+
+  private final HelixManager _helixManager;
+  private final String _tableNameWithType;
+  private boolean _allSegmentsLoaded;

Review comment:
       I see. Is it possible for a loaded segment to be reloaded?

##########
File path: pinot-core/src/test/java/org/apache/pinot/core/util/TableConfigUtilsTest.java
##########
@@ -1098,4 +1105,46 @@ public void testValidateUpsertConfig() {
       Assert.assertEquals(e.getMessage(), "The upsert table cannot have star-tree index.");
     }
   }
+
+  @Test
+  public void testValidatePartialUpsertConfig() {
+    Schema schema =
+        new Schema.SchemaBuilder().setSchemaName(TABLE_NAME).addSingleValueDimension("myCol1", FieldSpec.DataType.LONG)
+            .addSingleValueDimension("myCol2", FieldSpec.DataType.STRING)
+            .setPrimaryKeyColumns(Lists.newArrayList("myCol1")).build();
+
+    Map<String, String> streamConfigs = new HashMap<>();
+    streamConfigs.put("stream.kafka.consumer.type", "highLevel");
+    streamConfigs.put("streamType", "kafka");
+    streamConfigs.put("stream.kafka.topic.name", "test");
+    streamConfigs
+        .put("stream.kafka.decoder.class.name", "org.apache.pinot.plugin.stream.kafka.KafkaJSONMessageDecoder");
+    streamConfigs.put("stream.kafka.consumer.type", "simple");
+
+    TableConfig tableConfig = new TableConfigBuilder(TableType.REALTIME).setTableName(TABLE_NAME)
+        .setUpsertConfig(new UpsertConfig(UpsertConfig.Mode.PARTIAL)).setNullHandlingEnabled(false)
+        .setRoutingConfig(new RoutingConfig(null, null, RoutingConfig.STRICT_REPLICA_GROUP_INSTANCE_SELECTOR_TYPE))
+        .setStreamConfigs(streamConfigs).build();
+    try {
+      TableConfigUtils.validateUpsertConfig(tableConfig, schema);
+    } catch (Exception e) {
+      Assert.assertEquals(e.getMessage(), "NullValueHandling is required to be enabled for partial upsert tables.");
+    }
+
+    Map<String, UpsertConfig.Strategy> partialUpsertStratgies = new HashMap<>();
+    partialUpsertStratgies.put("myCol1", UpsertConfig.Strategy.INCREMENT);
+    try {
+      TableConfigUtils.validateUpsertConfig(tableConfig, schema);
+    } catch (Exception e) {
+      Assert.assertEquals(e.getMessage(), "INCREMENT merger cannot be applied to PK.");
+    }
+
+    partialUpsertStratgies = new HashMap<>();
+    partialUpsertStratgies.put("myCol2", UpsertConfig.Strategy.INCREMENT);
+    try {
+      TableConfigUtils.validateUpsertConfig(tableConfig, schema);
+    } catch (Exception e) {
+      Assert.assertEquals(e.getMessage(), "INCREMENT merger should be numeric data types.");

Review comment:
       One possible use case is the time counter increments.

##########
File path: pinot-segment-local/src/main/java/org/apache/pinot/segment/local/upsert/PartitionUpsertMetadataManager.java
##########
@@ -60,128 +64,175 @@
   private final String _tableNameWithType;
   private final int _partitionId;
   private final ServerMetrics _serverMetrics;
+  private final PartialUpsertHandler _partialUpsertHandler;
 
-  public PartitionUpsertMetadataManager(String tableNameWithType, int partitionId, ServerMetrics serverMetrics) {
+  // TODO(upset): consider an off-heap KV store to persist this index to improve the recovery speed.
+  @VisibleForTesting
+  final ConcurrentHashMap<PrimaryKey, RecordLocation> _primaryKeyToRecordLocationMap = new ConcurrentHashMap<>();
+
+  // Reused for reading previous record during partial upsert
+  private final GenericRow _reuse = new GenericRow();
+  // Stores the result of updateRecord()
+  private GenericRow _result;
+
+  public PartitionUpsertMetadataManager(String tableNameWithType, int partitionId, ServerMetrics serverMetrics,
+      @Nullable PartialUpsertHandler partialUpsertHandler) {
     _tableNameWithType = tableNameWithType;
     _partitionId = partitionId;
     _serverMetrics = serverMetrics;
+    _partialUpsertHandler = partialUpsertHandler;
   }
 
-  public ConcurrentHashMap<PrimaryKey, RecordLocation> getPrimaryKeyToRecordLocationMap() {
-    return _primaryKeyToRecordLocationMap;
+  /**
+   * Initializes the upsert metadata for the given immutable segment.
+   */
+  public void addSegment(ImmutableSegmentImpl immutableSegment, Iterator<RecordInfo> recordInfoIterator) {
+    addSegment(immutableSegment, recordInfoIterator, new ThreadSafeMutableRoaringBitmap());
   }
 
-  // TODO(upset): consider an off-heap KV store to persist this index to improve the recovery speed.
   @VisibleForTesting
-  final ConcurrentHashMap<PrimaryKey, RecordLocation> _primaryKeyToRecordLocationMap = new ConcurrentHashMap<>();
-
-  /**
-   * Initializes the upsert metadata for the given immutable segment, returns the valid doc ids for the segment.
-   */
-  public ThreadSafeMutableRoaringBitmap addSegment(String segmentName, Iterator<RecordInfo> recordInfoIterator) {
+  void addSegment(ImmutableSegmentImpl immutableSegment, Iterator<RecordInfo> recordInfoIterator,
+      ThreadSafeMutableRoaringBitmap validDocIds) {
+    String segmentName = immutableSegment.getSegmentName();
     LOGGER.info("Adding upsert metadata for segment: {}", segmentName);
+    immutableSegment.enableUpsert(this, validDocIds);
 
-    ThreadSafeMutableRoaringBitmap validDocIds = new ThreadSafeMutableRoaringBitmap();
     while (recordInfoIterator.hasNext()) {
       RecordInfo recordInfo = recordInfoIterator.next();
       _primaryKeyToRecordLocationMap.compute(recordInfo._primaryKey, (primaryKey, currentRecordLocation) -> {
         if (currentRecordLocation != null) {
           // Existing primary key
 
-          if (segmentName.equals(currentRecordLocation.getSegmentName())) {
-            // The current record location has the same segment name
-
-            // Update the record location when the new timestamp is greater than or equal to the current timestamp.
-            // There are 2 scenarios:
-            //   1. The current record location is pointing to the same segment (the segment being added). In this case,
-            //      we want to update the record location when there is a tie to keep the newer record. Note that the
-            //      record info iterator will return records with incremental doc ids.
-            //   2. The current record location is pointing to the old segment being replaced. This could happen when
-            //      committing a consuming segment, or reloading a completed segment. In this case, we want to update
-            //      the record location when there is a tie because the record locations should point to the new added
-            //      segment instead of the old segment being replaced. Also, do not update the valid doc ids for the old
-            //      segment because it has not been replaced yet.
+          // The current record is in the same segment
+          // Update the record location when there is a tie to keep the newer record. Note that the record info iterator
+          // will return records with incremental doc ids.
+          IndexSegment currentSegment = currentRecordLocation.getSegment();
+          if (immutableSegment == currentSegment) {
             if (recordInfo._timestamp >= currentRecordLocation.getTimestamp()) {
-              // Only update the valid doc ids for the new segment
-              if (validDocIds == currentRecordLocation.getValidDocIds()) {
-                validDocIds.remove(currentRecordLocation.getDocId());
-              }
+              validDocIds.remove(currentRecordLocation.getDocId());
               validDocIds.add(recordInfo._docId);
-              return new RecordLocation(segmentName, recordInfo._docId, recordInfo._timestamp, validDocIds);
+              return new RecordLocation(immutableSegment, recordInfo._docId, recordInfo._timestamp);
             } else {
               return currentRecordLocation;
             }
-          } else {
-            // The current record location is pointing to a different segment
-
-            // Update the record location when getting a newer timestamp, or the timestamp is the same as the current
-            // timestamp, but the segment has a larger sequence number (the segment is newer than the current segment).
-            if (recordInfo._timestamp > currentRecordLocation.getTimestamp() || (
-                recordInfo._timestamp == currentRecordLocation.getTimestamp()
-                    && LLCSegmentName.isLowLevelConsumerSegmentName(segmentName)
-                    && LLCSegmentName.isLowLevelConsumerSegmentName(currentRecordLocation.getSegmentName())
-                    && LLCSegmentName.getSequenceNumber(segmentName) > LLCSegmentName
-                    .getSequenceNumber(currentRecordLocation.getSegmentName()))) {
-              currentRecordLocation.getValidDocIds().remove(currentRecordLocation.getDocId());
+          }
+
+          // The current record is in an old segment being replaced
+          // This could happen when committing a consuming segment, or reloading a completed segment. In this case, we
+          // want to update the record location when there is a tie because the record locations should point to the new
+          // added segment instead of the old segment being replaced. Also, do not update the valid doc ids for the old
+          // segment because it has not been replaced yet.
+          String currentSegmentName = currentSegment.getSegmentName();
+          if (segmentName.equals(currentSegmentName)) {
+            if (recordInfo._timestamp >= currentRecordLocation.getTimestamp()) {
               validDocIds.add(recordInfo._docId);
-              return new RecordLocation(segmentName, recordInfo._docId, recordInfo._timestamp, validDocIds);
+              return new RecordLocation(immutableSegment, recordInfo._docId, recordInfo._timestamp);
             } else {
               return currentRecordLocation;
             }
           }
+
+          // The current record is in a different segment
+          // Update the record location when getting a newer timestamp, or the timestamp is the same as the current
+          // timestamp, but the segment has a larger sequence number (the segment is newer than the current segment).
+          if (recordInfo._timestamp > currentRecordLocation.getTimestamp() || (
+              recordInfo._timestamp == currentRecordLocation.getTimestamp() && LLCSegmentName
+                  .isLowLevelConsumerSegmentName(segmentName) && LLCSegmentName
+                  .isLowLevelConsumerSegmentName(currentSegmentName)
+                  && LLCSegmentName.getSequenceNumber(segmentName) > LLCSegmentName
+                  .getSequenceNumber(currentSegmentName))) {
+            assert currentSegment.getValidDocIds() != null;
+            currentSegment.getValidDocIds().remove(currentRecordLocation.getDocId());
+            validDocIds.add(recordInfo._docId);
+            return new RecordLocation(immutableSegment, recordInfo._docId, recordInfo._timestamp);
+          } else {
+            return currentRecordLocation;
+          }
         } else {
           // New primary key
           validDocIds.add(recordInfo._docId);
-          return new RecordLocation(segmentName, recordInfo._docId, recordInfo._timestamp, validDocIds);
+          return new RecordLocation(immutableSegment, recordInfo._docId, recordInfo._timestamp);
         }
       });
     }
     // Update metrics
     _serverMetrics.setValueOfPartitionGauge(_tableNameWithType, _partitionId, ServerGauge.UPSERT_PRIMARY_KEYS_COUNT,
         _primaryKeyToRecordLocationMap.size());
-    return validDocIds;
   }
 
   /**
-   * Updates the upsert metadata for a new consumed record in the given consuming segment.
+   * Updates the upsert metadata for a new consumed record in the given consuming segment. Returns the merged record if
+   * partial-upsert is enabled.
    */
-  public void updateRecord(String segmentName, RecordInfo recordInfo, ThreadSafeMutableRoaringBitmap validDocIds) {
+  public GenericRow updateRecord(IndexSegment segment, RecordInfo recordInfo, GenericRow record) {
+    // For partial-upsert, need to ensure all previous records are loaded before inserting new records.
+    if (_partialUpsertHandler != null) {
+      while (!_partialUpsertHandler.isAllSegmentsLoaded()) {
+        LOGGER
+            .info("Sleeping 1 second waiting for all segments loaded for partial-upsert table: {}", _tableNameWithType);
+        try {
+          //noinspection BusyWait
+          Thread.sleep(1000L);
+        } catch (InterruptedException e) {
+          throw new RuntimeException(e);
+        }
+      }
+    }
+
+    _result = record;
     _primaryKeyToRecordLocationMap.compute(recordInfo._primaryKey, (primaryKey, currentRecordLocation) -> {
       if (currentRecordLocation != null) {
         // Existing primary key
 
         // Update the record location when the new timestamp is greater than or equal to the current timestamp. Update
         // the record location when there is a tie to keep the newer record.
         if (recordInfo._timestamp >= currentRecordLocation.getTimestamp()) {
-          currentRecordLocation.getValidDocIds().remove(currentRecordLocation.getDocId());
-          validDocIds.add(recordInfo._docId);
-          return new RecordLocation(segmentName, recordInfo._docId, recordInfo._timestamp, validDocIds);
+          IndexSegment currentSegment = currentRecordLocation.getSegment();
+          if (_partialUpsertHandler != null) {
+            // Partial upsert
+            GenericRow previousRecord = currentSegment.getRecord(currentRecordLocation.getDocId(), _reuse);

Review comment:
       hmm what if the segment is expired in between?

##########
File path: pinot-segment-local/src/main/java/org/apache/pinot/segment/local/upsert/merger/IncrementMerger.java
##########
@@ -0,0 +1,44 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.segment.local.upsert.merger;
+
+public class IncrementMerger implements PartialUpsertMerger {
+  IncrementMerger() {

Review comment:
       ack




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

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



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


[GitHub] [incubator-pinot] chenboat commented on a change in pull request #6899: Add partial upsert config and mergers (WIP)

Posted by GitBox <gi...@apache.org>.
chenboat commented on a change in pull request #6899:
URL: https://github.com/apache/incubator-pinot/pull/6899#discussion_r634829027



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/indexsegment/mutable/MutableSegmentImpl.java
##########
@@ -499,6 +515,29 @@ private boolean isUpsertEnabled() {
     return _upsertMode != UpsertConfig.Mode.NONE;
   }
 
+  private boolean isPartialUpsertEnabled() {
+    return _upsertMode == UpsertConfig.Mode.PARTIAL;
+  }
+
+  private GenericRow lookupAndMerge(GenericRow incomingRow, int docId) {
+    // get primary key and timestamp for the incoming record.
+    GenericRow previousRow = new GenericRow();
+    PrimaryKey primaryKey = incomingRow.getPrimaryKey(_schema.getPrimaryKeyColumns());
+    Object timeValue = incomingRow.getValue(_timeColumnName);
+    Preconditions.checkArgument(timeValue instanceof Comparable, "time column shall be comparable");

Review comment:
       add the timeColumn and value to the log.




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

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



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


[GitHub] [incubator-pinot] chenboat commented on a change in pull request #6899: Add partial upsert config and mergers (WIP)

Posted by GitBox <gi...@apache.org>.
chenboat commented on a change in pull request #6899:
URL: https://github.com/apache/incubator-pinot/pull/6899#discussion_r634832351



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/indexsegment/mutable/MutableSegmentImpl.java
##########
@@ -499,6 +515,29 @@ private boolean isUpsertEnabled() {
     return _upsertMode != UpsertConfig.Mode.NONE;
   }
 
+  private boolean isPartialUpsertEnabled() {
+    return _upsertMode == UpsertConfig.Mode.PARTIAL;
+  }
+
+  private GenericRow lookupAndMerge(GenericRow incomingRow, int docId) {

Review comment:
       Add tests for this 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.

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] [incubator-pinot] Jackie-Jiang commented on a change in pull request #6899: Add partial upsert config and mergers

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on a change in pull request #6899:
URL: https://github.com/apache/incubator-pinot/pull/6899#discussion_r655586939



##########
File path: pinot-core/src/test/java/org/apache/pinot/core/util/TableConfigUtilsTest.java
##########
@@ -1098,4 +1105,46 @@ public void testValidateUpsertConfig() {
       Assert.assertEquals(e.getMessage(), "The upsert table cannot have star-tree index.");
     }
   }
+
+  @Test
+  public void testValidatePartialUpsertConfig() {
+    Schema schema =
+        new Schema.SchemaBuilder().setSchemaName(TABLE_NAME).addSingleValueDimension("myCol1", FieldSpec.DataType.LONG)
+            .addSingleValueDimension("myCol2", FieldSpec.DataType.STRING)
+            .setPrimaryKeyColumns(Lists.newArrayList("myCol1")).build();
+
+    Map<String, String> streamConfigs = new HashMap<>();
+    streamConfigs.put("stream.kafka.consumer.type", "highLevel");
+    streamConfigs.put("streamType", "kafka");
+    streamConfigs.put("stream.kafka.topic.name", "test");
+    streamConfigs
+        .put("stream.kafka.decoder.class.name", "org.apache.pinot.plugin.stream.kafka.KafkaJSONMessageDecoder");
+    streamConfigs.put("stream.kafka.consumer.type", "simple");
+
+    TableConfig tableConfig = new TableConfigBuilder(TableType.REALTIME).setTableName(TABLE_NAME)
+        .setUpsertConfig(new UpsertConfig(UpsertConfig.Mode.PARTIAL)).setNullHandlingEnabled(false)
+        .setRoutingConfig(new RoutingConfig(null, null, RoutingConfig.STRICT_REPLICA_GROUP_INSTANCE_SELECTOR_TYPE))
+        .setStreamConfigs(streamConfigs).build();
+    try {
+      TableConfigUtils.validateUpsertConfig(tableConfig, schema);
+    } catch (Exception e) {
+      Assert.assertEquals(e.getMessage(), "NullValueHandling is required to be enabled for partial upsert tables.");
+    }
+
+    Map<String, UpsertConfig.Strategy> partialUpsertStratgies = new HashMap<>();
+    partialUpsertStratgies.put("myCol1", UpsertConfig.Strategy.INCREMENT);
+    try {
+      TableConfigUtils.validateUpsertConfig(tableConfig, schema);
+    } catch (Exception e) {
+      Assert.assertEquals(e.getMessage(), "INCREMENT merger cannot be applied to PK.");
+    }
+
+    partialUpsertStratgies = new HashMap<>();
+    partialUpsertStratgies.put("myCol2", UpsertConfig.Strategy.INCREMENT);
+    try {
+      TableConfigUtils.validateUpsertConfig(tableConfig, schema);
+    } catch (Exception e) {
+      Assert.assertEquals(e.getMessage(), "INCREMENT merger should be numeric data types.");

Review comment:
       @yupeng9 The wait time is not really a time column, but a dimension column. Time column must be timestamp or datetime. There is no limit on the data type, as long as the value is stored as numeric type




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

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



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


[GitHub] [incubator-pinot] codecov-commenter edited a comment on pull request #6899: Add partial upsert config and mergers

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #6899:
URL: https://github.com/apache/incubator-pinot/pull/6899#issuecomment-853568285


   # [Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6899?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 [#6899](https://codecov.io/gh/apache/incubator-pinot/pull/6899?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (5e33df4) into [master](https://codecov.io/gh/apache/incubator-pinot/commit/a1c9b631381a25ddd6d3164d6a9ce337c3939b9f?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (a1c9b63) will **decrease** coverage by `8.08%`.
   > The diff coverage is `33.33%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/incubator-pinot/pull/6899/graphs/tree.svg?width=650&height=150&src=pr&token=4ibza2ugkz&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/incubator-pinot/pull/6899?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #6899      +/-   ##
   ============================================
   - Coverage     73.38%   65.30%   -8.09%     
     Complexity       12       12              
   ============================================
     Files          1453     1457       +4     
     Lines         72032    72168     +136     
     Branches      10427    10456      +29     
   ============================================
   - Hits          52863    47127    -5736     
   - Misses        15643    21624    +5981     
   + Partials       3526     3417     -109     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration | `?` | |
   | unittests | `65.30% <33.33%> (-0.11%)` | :arrow_down: |
   
   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/incubator-pinot/pull/6899?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...ata/manager/realtime/RealtimeTableDataManager.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9kYXRhL21hbmFnZXIvcmVhbHRpbWUvUmVhbHRpbWVUYWJsZURhdGFNYW5hZ2VyLmphdmE=) | `10.65% <0.00%> (-58.13%)` | :arrow_down: |
   | [...ocal/indexsegment/immutable/EmptyIndexSegment.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9pbmRleHNlZ21lbnQvaW1tdXRhYmxlL0VtcHR5SW5kZXhTZWdtZW50LmphdmE=) | `36.84% <ø> (ø)` | |
   | [...ocal/indexsegment/mutable/IntermediateSegment.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9pbmRleHNlZ21lbnQvbXV0YWJsZS9JbnRlcm1lZGlhdGVTZWdtZW50LmphdmE=) | `70.06% <ø> (ø)` | |
   | [...local/realtime/impl/geospatial/MutableH3Index.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9yZWFsdGltZS9pbXBsL2dlb3NwYXRpYWwvTXV0YWJsZUgzSW5kZXguamF2YQ==) | `93.33% <ø> (ø)` | |
   | [...mpl/invertedindex/RealtimeInvertedIndexReader.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9yZWFsdGltZS9pbXBsL2ludmVydGVkaW5kZXgvUmVhbHRpbWVJbnZlcnRlZEluZGV4UmVhZGVyLmphdmE=) | `100.00% <ø> (ø)` | |
   | [...ealtime/impl/nullvalue/MutableNullValueVector.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9yZWFsdGltZS9pbXBsL251bGx2YWx1ZS9NdXRhYmxlTnVsbFZhbHVlVmVjdG9yLmphdmE=) | `85.71% <ø> (-14.29%)` | :arrow_down: |
   | [...segment/index/readers/ValidDocIndexReaderImpl.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9zZWdtZW50L2luZGV4L3JlYWRlcnMvVmFsaWREb2NJbmRleFJlYWRlckltcGwuamF2YQ==) | `0.00% <ø> (-100.00%)` | :arrow_down: |
   | [...not/segment/local/upsert/PartialUpsertHandler.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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==) | `0.00% <0.00%> (ø)` | |
   | [...t/segment/local/upsert/merger/IncrementMerger.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC91cHNlcnQvbWVyZ2VyL0luY3JlbWVudE1lcmdlci5qYXZh) | `0.00% <0.00%> (ø)` | |
   | [...t/segment/local/upsert/merger/OverwriteMerger.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC91cHNlcnQvbWVyZ2VyL092ZXJ3cml0ZU1lcmdlci5qYXZh) | `0.00% <0.00%> (ø)` | |
   | ... and [368 more](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6899?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/incubator-pinot/pull/6899?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 [a1c9b63...5e33df4](https://codecov.io/gh/apache/incubator-pinot/pull/6899?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.

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] [incubator-pinot] codecov-commenter edited a comment on pull request #6899: Add partial upsert config and mergers

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #6899:
URL: https://github.com/apache/incubator-pinot/pull/6899#issuecomment-853568285






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

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



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


[GitHub] [incubator-pinot] deemoliu commented on a change in pull request #6899: Add partial upsert config and mergers

Posted by GitBox <gi...@apache.org>.
deemoliu commented on a change in pull request #6899:
URL: https://github.com/apache/incubator-pinot/pull/6899#discussion_r654042298



##########
File path: pinot-segment-local/src/test/java/org/apache/pinot/segment/local/upsert/PartialUpsertHandlerTest.java
##########
@@ -0,0 +1,81 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.segment.local.upsert;
+
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.helix.HelixManager;
+import org.apache.pinot.spi.config.table.UpsertConfig;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.mockito.Mockito;
+import org.testng.annotations.Test;
+
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertFalse;
+import static org.testng.Assert.assertTrue;
+
+
+public class PartialUpsertHandlerTest {
+
+  @Test
+  public void testMerge() {
+    HelixManager helixManager = Mockito.mock(HelixManager.class);
+    String realtimeTableName = "testTable_REALTIME";
+    Map<String, UpsertConfig.Strategy> partialUpsertStrategies = new HashMap<>();
+    partialUpsertStrategies.put("field1", UpsertConfig.Strategy.INCREMENT);

Review comment:
       @chenboat Current mergers are relatively simple, and null value handling logic are moved to handler part. the merger logic is covered in the handler test https://github.com/apache/incubator-pinot/pull/6899/commits/ec409cb739ffbe42b424003596cf9443a31d8ebf
   
   

##########
File path: pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java
##########
@@ -470,28 +466,32 @@ public void addExtraColumns(Schema newSchema) {
   @Override
   public boolean index(GenericRow row, @Nullable RowMetadata rowMetadata)
       throws IOException {
-    // Update dictionary first
-    updateDictionary(row);
-
-    // If metrics aggregation is enabled and if the dimension values were already seen, this will return existing docId,
-    // else this will return a new docId.
-    int docId = getOrCreateDocId();
-
     boolean canTakeMore;
-    if (docId == _numDocsIndexed) {
-      // New row
+    if (isUpsertEnabled()) {
+      row = handleUpsert(row, _numDocsIndexed);
+
+      updateDictionary(row);
       addNewRow(row);
       // Update number of documents indexed at last to make the latest row queryable
       canTakeMore = _numDocsIndexed++ < _capacity;
-
-      if (isUpsertEnabled()) {
-        handleUpsert(row, docId);
-      }
     } else {
-      Preconditions.checkArgument(!isUpsertEnabled(), "metrics aggregation cannot be used with upsert");
-      assert _aggregateMetrics;
-      aggregateMetrics(row, docId);
-      canTakeMore = true;
+      // Update dictionary first
+      updateDictionary(row);
+
+      // If metrics aggregation is enabled and if the dimension values were already seen, this will return existing

Review comment:
       gotcha, I can leave the comment here, and add some javadoc for getOrCreateDocId()  as well. 

##########
File path: pinot-segment-local/src/main/java/org/apache/pinot/segment/local/upsert/PartialUpsertHandler.java
##########
@@ -0,0 +1,141 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.segment.local.upsert;
+
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.helix.HelixDataAccessor;
+import org.apache.helix.HelixManager;
+import org.apache.helix.PropertyKey;
+import org.apache.helix.model.CurrentState;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.LiveInstance;
+import org.apache.pinot.segment.local.upsert.merger.PartialUpsertMerger;
+import org.apache.pinot.segment.local.upsert.merger.PartialUpsertMergerFactory;
+import org.apache.pinot.spi.config.table.UpsertConfig;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.utils.CommonConstants.Helix.StateModel.SegmentStateModel;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Handler for partial-upsert.
+ */
+public class PartialUpsertHandler {
+  private static final Logger LOGGER = LoggerFactory.getLogger(PartialUpsertHandler.class);
+
+  private final Map<String, PartialUpsertMerger> _mergers = new HashMap<>();
+
+  private final HelixManager _helixManager;
+  private final String _tableNameWithType;
+  private boolean _allSegmentsLoaded;
+
+  public PartialUpsertHandler(HelixManager helixManager, String tableNameWithType,
+      Map<String, UpsertConfig.Strategy> partialUpsertStrategies) {
+    _helixManager = helixManager;
+    _tableNameWithType = tableNameWithType;
+    for (Map.Entry<String, UpsertConfig.Strategy> entry : partialUpsertStrategies.entrySet()) {
+      _mergers.put(entry.getKey(), PartialUpsertMergerFactory.getMerger(entry.getValue()));
+    }
+  }
+
+  /**
+   * Returns {@code true} if all segments assigned to the current instance are loaded, {@code false} otherwise.
+   * Consuming segment should perform this check to ensure all previous records are loaded before inserting new records.
+   */
+  public synchronized boolean isAllSegmentsLoaded() {
+    if (_allSegmentsLoaded) {
+      return true;
+    }
+
+    HelixDataAccessor dataAccessor = _helixManager.getHelixDataAccessor();
+    PropertyKey.Builder keyBuilder = dataAccessor.keyBuilder();
+    IdealState idealState = dataAccessor.getProperty(keyBuilder.idealStates(_tableNameWithType));
+    if (idealState == null) {
+      LOGGER.warn("Failed to find ideal state for table: {}", _tableNameWithType);
+      return false;
+    }
+    String instanceName = _helixManager.getInstanceName();
+    LiveInstance liveInstance = dataAccessor.getProperty(keyBuilder.liveInstance(instanceName));
+    if (liveInstance == null) {
+      LOGGER.warn("Failed to find live instance for instance: {}", instanceName);
+      return false;
+    }
+    String sessionId = liveInstance.getEphemeralOwner();
+    CurrentState currentState =
+        dataAccessor.getProperty(keyBuilder.currentState(instanceName, sessionId, _tableNameWithType));
+    if (currentState == null) {
+      LOGGER.warn("Failed to find current state for instance: {}, sessionId: {}, table: {}", instanceName, sessionId,
+          _tableNameWithType);
+      return false;
+    }
+
+    // Check if ideal state and current state matches for all segments assigned to the current instance
+    Map<String, Map<String, String>> idealStatesMap = idealState.getRecord().getMapFields();
+    Map<String, String> currentStateMap = currentState.getPartitionStateMap();
+    for (Map.Entry<String, Map<String, String>> entry : idealStatesMap.entrySet()) {
+      String segmentName = entry.getKey();
+      Map<String, String> instanceStateMap = entry.getValue();
+      String expectedState = instanceStateMap.get(instanceName);
+      // Only track ONLINE segments assigned to the current instance
+      if (!SegmentStateModel.ONLINE.equals(expectedState)) {
+        continue;
+      }
+      String actualState = currentStateMap.get(segmentName);
+      if (!SegmentStateModel.ONLINE.equals(actualState)) {
+        if (SegmentStateModel.ERROR.equals(actualState)) {
+          LOGGER
+              .error("Find ERROR segment: {}, table: {}, expected: {}", segmentName, _tableNameWithType, expectedState);
+        } else {
+          LOGGER.info("Find unloaded segment: {}, table: {}, expected: {}, actual: {}", segmentName, _tableNameWithType,
+              expectedState, actualState);
+        }
+        return false;
+      }
+    }
+
+    LOGGER.info("All segments loaded for table: {}", _tableNameWithType);
+    _allSegmentsLoaded = true;
+    return true;
+  }
+
+  /**
+   * Merges 2 records and returns the merged record.
+   *
+   * @param previousRecord the last derived full record during ingestion.
+   * @param newRecord the new consumed record.
+   * @return a new row after merge
+   */
+  public GenericRow merge(GenericRow previousRecord, GenericRow newRecord) {
+    for (Map.Entry<String, PartialUpsertMerger> entry : _mergers.entrySet()) {
+      String column = entry.getKey();
+      if (!previousRecord.isNullValue(column)) {

Review comment:
       @chenboat after discuss with @Jackie-Jiang  and @yupeng9 we finalized to update merge interface from merge(row1, row2) to merge (value1, value2). With this change null value handling is moved to the handler and removed duplicated null value handling code in the mergers.




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

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



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


[GitHub] [incubator-pinot] yupeng9 commented on a change in pull request #6899: Add partial upsert config and mergers

Posted by GitBox <gi...@apache.org>.
yupeng9 commented on a change in pull request #6899:
URL: https://github.com/apache/incubator-pinot/pull/6899#discussion_r655598819



##########
File path: pinot-core/src/test/java/org/apache/pinot/core/util/TableConfigUtilsTest.java
##########
@@ -1098,4 +1105,46 @@ public void testValidateUpsertConfig() {
       Assert.assertEquals(e.getMessage(), "The upsert table cannot have star-tree index.");
     }
   }
+
+  @Test
+  public void testValidatePartialUpsertConfig() {
+    Schema schema =
+        new Schema.SchemaBuilder().setSchemaName(TABLE_NAME).addSingleValueDimension("myCol1", FieldSpec.DataType.LONG)
+            .addSingleValueDimension("myCol2", FieldSpec.DataType.STRING)
+            .setPrimaryKeyColumns(Lists.newArrayList("myCol1")).build();
+
+    Map<String, String> streamConfigs = new HashMap<>();
+    streamConfigs.put("stream.kafka.consumer.type", "highLevel");
+    streamConfigs.put("streamType", "kafka");
+    streamConfigs.put("stream.kafka.topic.name", "test");
+    streamConfigs
+        .put("stream.kafka.decoder.class.name", "org.apache.pinot.plugin.stream.kafka.KafkaJSONMessageDecoder");
+    streamConfigs.put("stream.kafka.consumer.type", "simple");
+
+    TableConfig tableConfig = new TableConfigBuilder(TableType.REALTIME).setTableName(TABLE_NAME)
+        .setUpsertConfig(new UpsertConfig(UpsertConfig.Mode.PARTIAL)).setNullHandlingEnabled(false)
+        .setRoutingConfig(new RoutingConfig(null, null, RoutingConfig.STRICT_REPLICA_GROUP_INSTANCE_SELECTOR_TYPE))
+        .setStreamConfigs(streamConfigs).build();
+    try {
+      TableConfigUtils.validateUpsertConfig(tableConfig, schema);
+    } catch (Exception e) {
+      Assert.assertEquals(e.getMessage(), "NullValueHandling is required to be enabled for partial upsert tables.");
+    }
+
+    Map<String, UpsertConfig.Strategy> partialUpsertStratgies = new HashMap<>();
+    partialUpsertStratgies.put("myCol1", UpsertConfig.Strategy.INCREMENT);
+    try {
+      TableConfigUtils.validateUpsertConfig(tableConfig, schema);
+    } catch (Exception e) {
+      Assert.assertEquals(e.getMessage(), "INCREMENT merger cannot be applied to PK.");
+    }
+
+    partialUpsertStratgies = new HashMap<>();
+    partialUpsertStratgies.put("myCol2", UpsertConfig.Strategy.INCREMENT);
+    try {
+      TableConfigUtils.validateUpsertConfig(tableConfig, schema);
+    } catch (Exception e) {
+      Assert.assertEquals(e.getMessage(), "INCREMENT merger should be numeric data types.");

Review comment:
       I see. So in this case, it's better to model it as a long type?




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

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



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


[GitHub] [incubator-pinot] deemoliu commented on a change in pull request #6899: Add partial upsert config and mergers

Posted by GitBox <gi...@apache.org>.
deemoliu commented on a change in pull request #6899:
URL: https://github.com/apache/incubator-pinot/pull/6899#discussion_r655733166



##########
File path: pinot-segment-local/src/main/java/org/apache/pinot/segment/local/upsert/PartitionUpsertMetadataManager.java
##########
@@ -60,128 +63,170 @@
   private final String _tableNameWithType;
   private final int _partitionId;
   private final ServerMetrics _serverMetrics;
+  private final PartialUpsertHandler _partialUpsertHandler;
 
-  public PartitionUpsertMetadataManager(String tableNameWithType, int partitionId, ServerMetrics serverMetrics) {
+  // TODO(upset): consider an off-heap KV store to persist this index to improve the recovery speed.
+  @VisibleForTesting
+  final ConcurrentHashMap<PrimaryKey, RecordLocation> _primaryKeyToRecordLocationMap = new ConcurrentHashMap<>();
+
+  // Reused for reading previous record during partial upsert
+  private final GenericRow _reuse = new GenericRow();
+  // Stores the result of updateRecord()
+  private GenericRow _result;
+
+  public PartitionUpsertMetadataManager(String tableNameWithType, int partitionId, ServerMetrics serverMetrics,
+      @Nullable PartialUpsertHandler partialUpsertHandler) {
     _tableNameWithType = tableNameWithType;
     _partitionId = partitionId;
     _serverMetrics = serverMetrics;
+    _partialUpsertHandler = partialUpsertHandler;
   }
 
-  public ConcurrentHashMap<PrimaryKey, RecordLocation> getPrimaryKeyToRecordLocationMap() {
-    return _primaryKeyToRecordLocationMap;
-  }
-
-  // TODO(upset): consider an off-heap KV store to persist this index to improve the recovery speed.
-  @VisibleForTesting
-  final ConcurrentHashMap<PrimaryKey, RecordLocation> _primaryKeyToRecordLocationMap = new ConcurrentHashMap<>();
-
   /**
-   * Initializes the upsert metadata for the given immutable segment, returns the valid doc ids for the segment.
+   * Initializes the upsert metadata for the given immutable segment.
    */
-  public ThreadSafeMutableRoaringBitmap addSegment(String segmentName, Iterator<RecordInfo> recordInfoIterator) {
+  public void addSegment(IndexSegment segment, Iterator<RecordInfo> recordInfoIterator) {
+    String segmentName = segment.getSegmentName();
     LOGGER.info("Adding upsert metadata for segment: {}", segmentName);
+    ThreadSafeMutableRoaringBitmap validDocIds = segment.getValidDocIds();
+    assert validDocIds != null;
 
-    ThreadSafeMutableRoaringBitmap validDocIds = new ThreadSafeMutableRoaringBitmap();
     while (recordInfoIterator.hasNext()) {
       RecordInfo recordInfo = recordInfoIterator.next();
       _primaryKeyToRecordLocationMap.compute(recordInfo._primaryKey, (primaryKey, currentRecordLocation) -> {
         if (currentRecordLocation != null) {
           // Existing primary key
 
-          if (segmentName.equals(currentRecordLocation.getSegmentName())) {
-            // The current record location has the same segment name
-
-            // Update the record location when the new timestamp is greater than or equal to the current timestamp.
-            // There are 2 scenarios:
-            //   1. The current record location is pointing to the same segment (the segment being added). In this case,
-            //      we want to update the record location when there is a tie to keep the newer record. Note that the
-            //      record info iterator will return records with incremental doc ids.
-            //   2. The current record location is pointing to the old segment being replaced. This could happen when
-            //      committing a consuming segment, or reloading a completed segment. In this case, we want to update
-            //      the record location when there is a tie because the record locations should point to the new added
-            //      segment instead of the old segment being replaced. Also, do not update the valid doc ids for the old
-            //      segment because it has not been replaced yet.
+          // The current record is in the same segment
+          // Update the record location when there is a tie to keep the newer record. Note that the record info iterator
+          // will return records with incremental doc ids.
+          IndexSegment currentSegment = currentRecordLocation.getSegment();
+          if (segment == currentSegment) {
             if (recordInfo._timestamp >= currentRecordLocation.getTimestamp()) {
-              // Only update the valid doc ids for the new segment
-              if (validDocIds == currentRecordLocation.getValidDocIds()) {
-                validDocIds.remove(currentRecordLocation.getDocId());
-              }
+              validDocIds.remove(currentRecordLocation.getDocId());
               validDocIds.add(recordInfo._docId);
-              return new RecordLocation(segmentName, recordInfo._docId, recordInfo._timestamp, validDocIds);
+              return new RecordLocation(segment, recordInfo._docId, recordInfo._timestamp);
             } else {
               return currentRecordLocation;
             }
-          } else {
-            // The current record location is pointing to a different segment
-
-            // Update the record location when getting a newer timestamp, or the timestamp is the same as the current
-            // timestamp, but the segment has a larger sequence number (the segment is newer than the current segment).
-            if (recordInfo._timestamp > currentRecordLocation.getTimestamp() || (
-                recordInfo._timestamp == currentRecordLocation.getTimestamp()
-                    && LLCSegmentName.isLowLevelConsumerSegmentName(segmentName)
-                    && LLCSegmentName.isLowLevelConsumerSegmentName(currentRecordLocation.getSegmentName())
-                    && LLCSegmentName.getSequenceNumber(segmentName) > LLCSegmentName
-                    .getSequenceNumber(currentRecordLocation.getSegmentName()))) {
-              currentRecordLocation.getValidDocIds().remove(currentRecordLocation.getDocId());
+          }
+
+          // The current record is in an old segment being replaced

Review comment:
       @chenboat This case happens when reloading a completed segment. The old segment is being replaced.
   cc'ed @Jackie-Jiang 
   




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

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



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


[GitHub] [incubator-pinot] deemoliu commented on a change in pull request #6899: Add partial upsert config and mergers

Posted by GitBox <gi...@apache.org>.
deemoliu commented on a change in pull request #6899:
URL: https://github.com/apache/incubator-pinot/pull/6899#discussion_r655736134



##########
File path: pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java
##########
@@ -470,28 +466,32 @@ public void addExtraColumns(Schema newSchema) {
   @Override
   public boolean index(GenericRow row, @Nullable RowMetadata rowMetadata)
       throws IOException {
-    // Update dictionary first
-    updateDictionary(row);
-
-    // If metrics aggregation is enabled and if the dimension values were already seen, this will return existing docId,
-    // else this will return a new docId.
-    int docId = getOrCreateDocId();
-
     boolean canTakeMore;
-    if (docId == _numDocsIndexed) {
-      // New row
+    if (isUpsertEnabled()) {
+      row = handleUpsert(row, _numDocsIndexed);
+
+      updateDictionary(row);
       addNewRow(row);
       // Update number of documents indexed at last to make the latest row queryable
       canTakeMore = _numDocsIndexed++ < _capacity;
-
-      if (isUpsertEnabled()) {
-        handleUpsert(row, docId);
-      }
     } else {
-      Preconditions.checkArgument(!isUpsertEnabled(), "metrics aggregation cannot be used with upsert");
-      assert _aggregateMetrics;
-      aggregateMetrics(row, docId);
-      canTakeMore = true;
+      // Update dictionary first
+      updateDictionary(row);
+
+      // If metrics aggregation is enabled and if the dimension values were already seen, this will return existing
+      // docId, else this will return a new docId.
+      int docId = getOrCreateDocId();
+
+      if (docId == _numDocsIndexed) {
+        // New row
+        addNewRow(row);
+        // Update number of documents indexed at last to make the latest row queryable
+        canTakeMore = _numDocsIndexed++ < _capacity;
+      } else {
+        assert _aggregateMetrics;

Review comment:
       gotcha, added a preCondition check 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.

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] [incubator-pinot] deemoliu commented on a change in pull request #6899: Add partial upsert config and mergers

Posted by GitBox <gi...@apache.org>.
deemoliu commented on a change in pull request #6899:
URL: https://github.com/apache/incubator-pinot/pull/6899#discussion_r655722565



##########
File path: pinot-segment-local/src/main/java/org/apache/pinot/segment/local/upsert/PartitionUpsertMetadataManager.java
##########
@@ -60,128 +63,170 @@
   private final String _tableNameWithType;
   private final int _partitionId;
   private final ServerMetrics _serverMetrics;
+  private final PartialUpsertHandler _partialUpsertHandler;
 
-  public PartitionUpsertMetadataManager(String tableNameWithType, int partitionId, ServerMetrics serverMetrics) {
+  // TODO(upset): consider an off-heap KV store to persist this index to improve the recovery speed.

Review comment:
       the index here means PK to recordLocation mapping. cc'ed @Jackie-Jiang i re-phrased this comment a bit. 




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

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



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


[GitHub] [incubator-pinot] deemoliu commented on a change in pull request #6899: Add partial upsert config and mergers

Posted by GitBox <gi...@apache.org>.
deemoliu commented on a change in pull request #6899:
URL: https://github.com/apache/incubator-pinot/pull/6899#discussion_r650307998



##########
File path: pinot-segment-local/src/main/java/org/apache/pinot/segment/local/upsert/merger/PartialUpsertMergerFactory.java
##########
@@ -0,0 +1,41 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.segment.local.upsert.merger;
+
+import org.apache.pinot.spi.config.table.UpsertConfig;
+
+
+public class PartialUpsertMergerFactory {
+  private PartialUpsertMergerFactory() {
+  }
+
+  private static final OverwriteMerger OVERWRITE_MERGER = new OverwriteMerger();
+  private static final IncrementMerger INCREMENT_MERGER = new IncrementMerger();
+
+  public static PartialUpsertMerger getMerger(UpsertConfig.Strategy strategy) {
+    switch (strategy) {
+      case OVERWRITE:
+        return OVERWRITE_MERGER;
+      case INCREMENT:
+        return INCREMENT_MERGER;

Review comment:
       hi @yupeng9 i added one merger tests here for null values https://github.com/apache/incubator-pinot/pull/6899/commits/ec409cb739ffbe42b424003596cf9443a31d8ebf

##########
File path: pinot-spi/src/main/java/org/apache/pinot/spi/config/table/UpsertConfig.java
##########
@@ -30,16 +32,37 @@
     FULL, PARTIAL, NONE
   }
 
+  public enum Strategy {
+    OVERWRITE, INCREMENT

Review comment:
       gotcha resolved in https://github.com/apache/incubator-pinot/pull/6899/commits/61356a46850d3c0eeb9313845ec77d10769486ba

##########
File path: pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/TableConfigUtils.java
##########
@@ -350,6 +357,36 @@ public static void validateUpsertConfig(TableConfig tableConfig, Schema schema)
             .getIndexingConfig().isEnableDefaultStarTree(), "The upsert table cannot have star-tree index.");
   }
 
+  /**
+   * Validates the partial upsert-related configurations
+   *  - INCREMENT merger cannot be applied to PK.
+   *  - INCREMENT merger should be numeric data types.
+   *  - enforce nullValueHandling for partial upsert tables.
+   */
+  private static void validatePartialUpsertStrategies(Schema schema, TableConfig tableConfig) {
+    if (tableConfig.getUpsertMode() != UpsertConfig.Mode.PARTIAL) {
+      return;
+    }
+
+    Preconditions.checkState(tableConfig.getIndexingConfig().isNullHandlingEnabled(),
+        "NullValueHandling is required to be enabled for partial upsert tables.");
+
+    Map<String, UpsertConfig.Strategy> partialUpsertStrategies =
+        tableConfig.getUpsertConfig().getPartialUpsertStrategies();
+
+    for (Map.Entry<String, UpsertConfig.Strategy> entry : partialUpsertStrategies.entrySet()) {
+      Set<FieldSpec.DataType> numericsDataType = new HashSet<>(Arrays.asList(INT, LONG, FLOAT, DOUBLE));

Review comment:
       thanks @yupeng9 ,  I added one more validation rules to avoid date time field with numeric datatype get "incremented". ​
   https://github.com/apache/incubator-pinot/pull/6899/commits/372e8e639b18418917f0e965cc9b6e2f189c7c02
   

##########
File path: pinot-core/src/test/java/org/apache/pinot/core/util/TableConfigUtilsTest.java
##########
@@ -984,17 +989,19 @@ public void testValidateIndexingConfig() {
       // Expected
     }
 
-    tableConfig = new TableConfigBuilder(TableType.OFFLINE).setTableName(TABLE_NAME).setVarLengthDictionaryColumns(Arrays.asList("intCol")).
-        build();
+    tableConfig = new TableConfigBuilder(TableType.OFFLINE).setTableName(TABLE_NAME)
+        .setVarLengthDictionaryColumns(Arrays.asList("intCol")).

Review comment:
       Thanks @yupeng9 @Jackie-Jiang, my IDE formatter might not be defined correctly. Is there any guide on this?

##########
File path: pinot-core/src/test/java/org/apache/pinot/core/util/TableConfigUtilsTest.java
##########
@@ -1098,4 +1105,46 @@ public void testValidateUpsertConfig() {
       Assert.assertEquals(e.getMessage(), "The upsert table cannot have star-tree index.");
     }
   }
+
+  @Test
+  public void testValidatePartialUpsertConfig() {
+    Schema schema =
+        new Schema.SchemaBuilder().setSchemaName(TABLE_NAME).addSingleValueDimension("myCol1", FieldSpec.DataType.LONG)
+            .addSingleValueDimension("myCol2", FieldSpec.DataType.STRING)
+            .setPrimaryKeyColumns(Lists.newArrayList("myCol1")).build();
+
+    Map<String, String> streamConfigs = new HashMap<>();
+    streamConfigs.put("stream.kafka.consumer.type", "highLevel");
+    streamConfigs.put("streamType", "kafka");
+    streamConfigs.put("stream.kafka.topic.name", "test");
+    streamConfigs
+        .put("stream.kafka.decoder.class.name", "org.apache.pinot.plugin.stream.kafka.KafkaJSONMessageDecoder");
+    streamConfigs.put("stream.kafka.consumer.type", "simple");
+
+    TableConfig tableConfig = new TableConfigBuilder(TableType.REALTIME).setTableName(TABLE_NAME)
+        .setUpsertConfig(new UpsertConfig(UpsertConfig.Mode.PARTIAL)).setNullHandlingEnabled(false)
+        .setRoutingConfig(new RoutingConfig(null, null, RoutingConfig.STRICT_REPLICA_GROUP_INSTANCE_SELECTOR_TYPE))
+        .setStreamConfigs(streamConfigs).build();
+    try {
+      TableConfigUtils.validateUpsertConfig(tableConfig, schema);
+    } catch (Exception e) {
+      Assert.assertEquals(e.getMessage(), "NullValueHandling is required to be enabled for partial upsert tables.");
+    }
+
+    Map<String, UpsertConfig.Strategy> partialUpsertStratgies = new HashMap<>();
+    partialUpsertStratgies.put("myCol1", UpsertConfig.Strategy.INCREMENT);
+    try {
+      TableConfigUtils.validateUpsertConfig(tableConfig, schema);
+    } catch (Exception e) {
+      Assert.assertEquals(e.getMessage(), "INCREMENT merger cannot be applied to PK.");
+    }
+
+    partialUpsertStratgies = new HashMap<>();
+    partialUpsertStratgies.put("myCol2", UpsertConfig.Strategy.INCREMENT);
+    try {
+      TableConfigUtils.validateUpsertConfig(tableConfig, schema);
+    } catch (Exception e) {
+      Assert.assertEquals(e.getMessage(), "INCREMENT merger should be numeric data types.");

Review comment:
       Thanks @yupeng9 for pointing this out. I don't see a use case that requires increment on timestamp for now. Please let me know there is any use case require this.
   Since time type can be Numeric Datatype, I added one more validation rules to avoid Numeric timestamp value get incremented.
   




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

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



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


[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #6899: Add partial upsert config and mergers

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on a change in pull request #6899:
URL: https://github.com/apache/incubator-pinot/pull/6899#discussion_r656358418



##########
File path: pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java
##########
@@ -470,28 +466,32 @@ public void addExtraColumns(Schema newSchema) {
   @Override
   public boolean index(GenericRow row, @Nullable RowMetadata rowMetadata)
       throws IOException {
-    // Update dictionary first
-    updateDictionary(row);
-
-    // If metrics aggregation is enabled and if the dimension values were already seen, this will return existing docId,
-    // else this will return a new docId.
-    int docId = getOrCreateDocId();
-
     boolean canTakeMore;
-    if (docId == _numDocsIndexed) {
-      // New row
+    if (isUpsertEnabled()) {
+      row = handleUpsert(row, _numDocsIndexed);
+
+      updateDictionary(row);
       addNewRow(row);
       // Update number of documents indexed at last to make the latest row queryable
       canTakeMore = _numDocsIndexed++ < _capacity;
-
-      if (isUpsertEnabled()) {
-        handleUpsert(row, docId);
-      }
     } else {
-      Preconditions.checkArgument(!isUpsertEnabled(), "metrics aggregation cannot be used with upsert");
-      assert _aggregateMetrics;
-      aggregateMetrics(row, docId);
-      canTakeMore = true;
+      // Update dictionary first
+      updateDictionary(row);
+
+      // If metrics aggregation is enabled and if the dimension values were already seen, this will return existing
+      // docId, else this will return a new docId.
+      int docId = getOrCreateDocId();
+
+      if (docId == _numDocsIndexed) {
+        // New row
+        addNewRow(row);
+        // Update number of documents indexed at last to make the latest row queryable
+        canTakeMore = _numDocsIndexed++ < _capacity;
+      } else {
+        assert _aggregateMetrics;

Review comment:
       @deemoliu Please remove this pre-condition check. The check is already performed in the constructor, and the assert here is just for readability




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

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



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


[GitHub] [incubator-pinot] chenboat commented on a change in pull request #6899: Add partial upsert config and mergers

Posted by GitBox <gi...@apache.org>.
chenboat commented on a change in pull request #6899:
URL: https://github.com/apache/incubator-pinot/pull/6899#discussion_r654067697



##########
File path: pinot-segment-local/src/main/java/org/apache/pinot/segment/local/upsert/PartitionUpsertMetadataManager.java
##########
@@ -60,128 +63,170 @@
   private final String _tableNameWithType;
   private final int _partitionId;
   private final ServerMetrics _serverMetrics;
+  private final PartialUpsertHandler _partialUpsertHandler;
 
-  public PartitionUpsertMetadataManager(String tableNameWithType, int partitionId, ServerMetrics serverMetrics) {
+  // TODO(upset): consider an off-heap KV store to persist this index to improve the recovery speed.

Review comment:
       s/upset/upsert/

##########
File path: pinot-segment-local/src/main/java/org/apache/pinot/segment/local/upsert/PartialUpsertHandler.java
##########
@@ -0,0 +1,141 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.segment.local.upsert;
+
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.helix.HelixDataAccessor;
+import org.apache.helix.HelixManager;
+import org.apache.helix.PropertyKey;
+import org.apache.helix.model.CurrentState;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.LiveInstance;
+import org.apache.pinot.segment.local.upsert.merger.PartialUpsertMerger;
+import org.apache.pinot.segment.local.upsert.merger.PartialUpsertMergerFactory;
+import org.apache.pinot.spi.config.table.UpsertConfig;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.utils.CommonConstants.Helix.StateModel.SegmentStateModel;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Handler for partial-upsert.
+ */
+public class PartialUpsertHandler {
+  private static final Logger LOGGER = LoggerFactory.getLogger(PartialUpsertHandler.class);
+
+  private final Map<String, PartialUpsertMerger> _mergers = new HashMap<>();
+
+  private final HelixManager _helixManager;
+  private final String _tableNameWithType;
+  private boolean _allSegmentsLoaded;
+
+  public PartialUpsertHandler(HelixManager helixManager, String tableNameWithType,
+      Map<String, UpsertConfig.Strategy> partialUpsertStrategies) {
+    _helixManager = helixManager;
+    _tableNameWithType = tableNameWithType;
+    for (Map.Entry<String, UpsertConfig.Strategy> entry : partialUpsertStrategies.entrySet()) {
+      _mergers.put(entry.getKey(), PartialUpsertMergerFactory.getMerger(entry.getValue()));
+    }
+  }
+
+  /**
+   * Returns {@code true} if all segments assigned to the current instance are loaded, {@code false} otherwise.
+   * Consuming segment should perform this check to ensure all previous records are loaded before inserting new records.
+   */
+  public synchronized boolean isAllSegmentsLoaded() {
+    if (_allSegmentsLoaded) {
+      return true;
+    }
+
+    HelixDataAccessor dataAccessor = _helixManager.getHelixDataAccessor();
+    PropertyKey.Builder keyBuilder = dataAccessor.keyBuilder();
+    IdealState idealState = dataAccessor.getProperty(keyBuilder.idealStates(_tableNameWithType));
+    if (idealState == null) {
+      LOGGER.warn("Failed to find ideal state for table: {}", _tableNameWithType);
+      return false;
+    }
+    String instanceName = _helixManager.getInstanceName();
+    LiveInstance liveInstance = dataAccessor.getProperty(keyBuilder.liveInstance(instanceName));
+    if (liveInstance == null) {
+      LOGGER.warn("Failed to find live instance for instance: {}", instanceName);
+      return false;
+    }
+    String sessionId = liveInstance.getEphemeralOwner();
+    CurrentState currentState =
+        dataAccessor.getProperty(keyBuilder.currentState(instanceName, sessionId, _tableNameWithType));
+    if (currentState == null) {
+      LOGGER.warn("Failed to find current state for instance: {}, sessionId: {}, table: {}", instanceName, sessionId,
+          _tableNameWithType);
+      return false;
+    }
+
+    // Check if ideal state and current state matches for all segments assigned to the current instance
+    Map<String, Map<String, String>> idealStatesMap = idealState.getRecord().getMapFields();
+    Map<String, String> currentStateMap = currentState.getPartitionStateMap();
+    for (Map.Entry<String, Map<String, String>> entry : idealStatesMap.entrySet()) {
+      String segmentName = entry.getKey();
+      Map<String, String> instanceStateMap = entry.getValue();
+      String expectedState = instanceStateMap.get(instanceName);
+      // Only track ONLINE segments assigned to the current instance
+      if (!SegmentStateModel.ONLINE.equals(expectedState)) {
+        continue;
+      }
+      String actualState = currentStateMap.get(segmentName);
+      if (!SegmentStateModel.ONLINE.equals(actualState)) {
+        if (SegmentStateModel.ERROR.equals(actualState)) {
+          LOGGER
+              .error("Find ERROR segment: {}, table: {}, expected: {}", segmentName, _tableNameWithType, expectedState);
+        } else {
+          LOGGER.info("Find unloaded segment: {}, table: {}, expected: {}, actual: {}", segmentName, _tableNameWithType,
+              expectedState, actualState);
+        }
+        return false;
+      }
+    }
+
+    LOGGER.info("All segments loaded for table: {}", _tableNameWithType);
+    _allSegmentsLoaded = true;
+    return true;
+  }
+
+  /**
+   * Merges 2 records and returns the merged record.
+   *
+   * @param previousRecord the last derived full record during ingestion.
+   * @param newRecord the new consumed record.
+   * @return a new row after merge
+   */
+  public GenericRow merge(GenericRow previousRecord, GenericRow newRecord) {
+    for (Map.Entry<String, PartialUpsertMerger> entry : _mergers.entrySet()) {
+      String column = entry.getKey();
+      if (!previousRecord.isNullValue(column)) {

Review comment:
       should it be part of a merge strategy to deal with Null value handling in two records? Why this logic appear here rather than delegating to each mergeStrategy?

##########
File path: pinot-segment-local/src/main/java/org/apache/pinot/segment/local/upsert/PartitionUpsertMetadataManager.java
##########
@@ -60,128 +63,170 @@
   private final String _tableNameWithType;
   private final int _partitionId;
   private final ServerMetrics _serverMetrics;
+  private final PartialUpsertHandler _partialUpsertHandler;
 
-  public PartitionUpsertMetadataManager(String tableNameWithType, int partitionId, ServerMetrics serverMetrics) {
+  // TODO(upset): consider an off-heap KV store to persist this index to improve the recovery speed.

Review comment:
       which index? can you be more specific?

##########
File path: pinot-segment-local/src/main/java/org/apache/pinot/segment/local/upsert/PartitionUpsertMetadataManager.java
##########
@@ -60,128 +63,170 @@
   private final String _tableNameWithType;
   private final int _partitionId;
   private final ServerMetrics _serverMetrics;
+  private final PartialUpsertHandler _partialUpsertHandler;
 
-  public PartitionUpsertMetadataManager(String tableNameWithType, int partitionId, ServerMetrics serverMetrics) {
+  // TODO(upset): consider an off-heap KV store to persist this index to improve the recovery speed.
+  @VisibleForTesting
+  final ConcurrentHashMap<PrimaryKey, RecordLocation> _primaryKeyToRecordLocationMap = new ConcurrentHashMap<>();
+
+  // Reused for reading previous record during partial upsert
+  private final GenericRow _reuse = new GenericRow();
+  // Stores the result of updateRecord()
+  private GenericRow _result;
+
+  public PartitionUpsertMetadataManager(String tableNameWithType, int partitionId, ServerMetrics serverMetrics,
+      @Nullable PartialUpsertHandler partialUpsertHandler) {
     _tableNameWithType = tableNameWithType;
     _partitionId = partitionId;
     _serverMetrics = serverMetrics;
+    _partialUpsertHandler = partialUpsertHandler;
   }
 
-  public ConcurrentHashMap<PrimaryKey, RecordLocation> getPrimaryKeyToRecordLocationMap() {
-    return _primaryKeyToRecordLocationMap;
-  }
-
-  // TODO(upset): consider an off-heap KV store to persist this index to improve the recovery speed.
-  @VisibleForTesting
-  final ConcurrentHashMap<PrimaryKey, RecordLocation> _primaryKeyToRecordLocationMap = new ConcurrentHashMap<>();
-
   /**
-   * Initializes the upsert metadata for the given immutable segment, returns the valid doc ids for the segment.
+   * Initializes the upsert metadata for the given immutable segment.
    */
-  public ThreadSafeMutableRoaringBitmap addSegment(String segmentName, Iterator<RecordInfo> recordInfoIterator) {
+  public void addSegment(IndexSegment segment, Iterator<RecordInfo> recordInfoIterator) {
+    String segmentName = segment.getSegmentName();
     LOGGER.info("Adding upsert metadata for segment: {}", segmentName);
+    ThreadSafeMutableRoaringBitmap validDocIds = segment.getValidDocIds();
+    assert validDocIds != null;
 
-    ThreadSafeMutableRoaringBitmap validDocIds = new ThreadSafeMutableRoaringBitmap();
     while (recordInfoIterator.hasNext()) {
       RecordInfo recordInfo = recordInfoIterator.next();
       _primaryKeyToRecordLocationMap.compute(recordInfo._primaryKey, (primaryKey, currentRecordLocation) -> {
         if (currentRecordLocation != null) {
           // Existing primary key
 
-          if (segmentName.equals(currentRecordLocation.getSegmentName())) {
-            // The current record location has the same segment name
-
-            // Update the record location when the new timestamp is greater than or equal to the current timestamp.
-            // There are 2 scenarios:
-            //   1. The current record location is pointing to the same segment (the segment being added). In this case,
-            //      we want to update the record location when there is a tie to keep the newer record. Note that the
-            //      record info iterator will return records with incremental doc ids.
-            //   2. The current record location is pointing to the old segment being replaced. This could happen when
-            //      committing a consuming segment, or reloading a completed segment. In this case, we want to update
-            //      the record location when there is a tie because the record locations should point to the new added
-            //      segment instead of the old segment being replaced. Also, do not update the valid doc ids for the old
-            //      segment because it has not been replaced yet.
+          // The current record is in the same segment
+          // Update the record location when there is a tie to keep the newer record. Note that the record info iterator
+          // will return records with incremental doc ids.
+          IndexSegment currentSegment = currentRecordLocation.getSegment();
+          if (segment == currentSegment) {
             if (recordInfo._timestamp >= currentRecordLocation.getTimestamp()) {
-              // Only update the valid doc ids for the new segment
-              if (validDocIds == currentRecordLocation.getValidDocIds()) {
-                validDocIds.remove(currentRecordLocation.getDocId());
-              }
+              validDocIds.remove(currentRecordLocation.getDocId());
               validDocIds.add(recordInfo._docId);
-              return new RecordLocation(segmentName, recordInfo._docId, recordInfo._timestamp, validDocIds);
+              return new RecordLocation(segment, recordInfo._docId, recordInfo._timestamp);
             } else {
               return currentRecordLocation;
             }
-          } else {
-            // The current record location is pointing to a different segment
-
-            // Update the record location when getting a newer timestamp, or the timestamp is the same as the current
-            // timestamp, but the segment has a larger sequence number (the segment is newer than the current segment).
-            if (recordInfo._timestamp > currentRecordLocation.getTimestamp() || (
-                recordInfo._timestamp == currentRecordLocation.getTimestamp()
-                    && LLCSegmentName.isLowLevelConsumerSegmentName(segmentName)
-                    && LLCSegmentName.isLowLevelConsumerSegmentName(currentRecordLocation.getSegmentName())
-                    && LLCSegmentName.getSequenceNumber(segmentName) > LLCSegmentName
-                    .getSequenceNumber(currentRecordLocation.getSegmentName()))) {
-              currentRecordLocation.getValidDocIds().remove(currentRecordLocation.getDocId());
+          }
+
+          // The current record is in an old segment being replaced

Review comment:
       What do you mean a segment being replaced? is it being deleted due to retention reasons?

##########
File path: pinot-segment-local/src/main/java/org/apache/pinot/segment/local/upsert/merger/PartialUpsertMerger.java
##########
@@ -0,0 +1,30 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.segment.local.upsert.merger;
+
+public interface PartialUpsertMerger {
+  /**
+   * Handle partial upsert merge.
+   *
+   * @param previousValue the last derived full record during ingestion.
+   * @param currentValue the new consumed record.

Review comment:
       Does the current record need to be full? In general, I do not understand why we should even have this constraint that either the previous or currentValue should be a full record or not. A merger should take care of all cases.

##########
File path: pinot-segment-local/src/test/java/org/apache/pinot/segment/local/upsert/PartitionUpsertMetadataManagerTest.java
##########
@@ -44,36 +46,36 @@
   @Test
   public void testAddSegment() {
     PartitionUpsertMetadataManager upsertMetadataManager =
-        new PartitionUpsertMetadataManager(REALTIME_TABLE_NAME, 0, Mockito.mock(ServerMetrics.class));
-    Map<PrimaryKey, RecordLocation> recordLocationMap = upsertMetadataManager.getPrimaryKeyToRecordLocationMap();
+        new PartitionUpsertMetadataManager(REALTIME_TABLE_NAME, 0, mock(ServerMetrics.class), null);

Review comment:
       Can you add tests in this class about PartialUpdate?




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

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



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


[GitHub] [incubator-pinot] deemoliu commented on a change in pull request #6899: Add partial upsert config and mergers

Posted by GitBox <gi...@apache.org>.
deemoliu commented on a change in pull request #6899:
URL: https://github.com/apache/incubator-pinot/pull/6899#discussion_r655734204



##########
File path: pinot-segment-local/src/main/java/org/apache/pinot/segment/local/upsert/merger/PartialUpsertMerger.java
##########
@@ -0,0 +1,30 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.segment.local.upsert.merger;
+
+public interface PartialUpsertMerger {
+  /**
+   * Handle partial upsert merge.
+   *
+   * @param previousValue the last derived full record during ingestion.
+   * @param currentValue the new consumed record.

Review comment:
       thanks for pointing this out. The javadoc is not correct. I updated the javadoc for this function.




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

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



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


[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #6899: Add partial upsert config and mergers

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on a change in pull request #6899:
URL: https://github.com/apache/incubator-pinot/pull/6899#discussion_r651130777



##########
File path: pinot-core/src/test/java/org/apache/pinot/core/util/TableConfigUtilsTest.java
##########
@@ -1098,4 +1105,46 @@ public void testValidateUpsertConfig() {
       Assert.assertEquals(e.getMessage(), "The upsert table cannot have star-tree index.");
     }
   }
+
+  @Test
+  public void testValidatePartialUpsertConfig() {
+    Schema schema =
+        new Schema.SchemaBuilder().setSchemaName(TABLE_NAME).addSingleValueDimension("myCol1", FieldSpec.DataType.LONG)
+            .addSingleValueDimension("myCol2", FieldSpec.DataType.STRING)
+            .setPrimaryKeyColumns(Lists.newArrayList("myCol1")).build();
+
+    Map<String, String> streamConfigs = new HashMap<>();
+    streamConfigs.put("stream.kafka.consumer.type", "highLevel");
+    streamConfigs.put("streamType", "kafka");
+    streamConfigs.put("stream.kafka.topic.name", "test");
+    streamConfigs
+        .put("stream.kafka.decoder.class.name", "org.apache.pinot.plugin.stream.kafka.KafkaJSONMessageDecoder");
+    streamConfigs.put("stream.kafka.consumer.type", "simple");
+
+    TableConfig tableConfig = new TableConfigBuilder(TableType.REALTIME).setTableName(TABLE_NAME)
+        .setUpsertConfig(new UpsertConfig(UpsertConfig.Mode.PARTIAL)).setNullHandlingEnabled(false)
+        .setRoutingConfig(new RoutingConfig(null, null, RoutingConfig.STRICT_REPLICA_GROUP_INSTANCE_SELECTOR_TYPE))
+        .setStreamConfigs(streamConfigs).build();
+    try {
+      TableConfigUtils.validateUpsertConfig(tableConfig, schema);
+    } catch (Exception e) {
+      Assert.assertEquals(e.getMessage(), "NullValueHandling is required to be enabled for partial upsert tables.");
+    }
+
+    Map<String, UpsertConfig.Strategy> partialUpsertStratgies = new HashMap<>();
+    partialUpsertStratgies.put("myCol1", UpsertConfig.Strategy.INCREMENT);
+    try {
+      TableConfigUtils.validateUpsertConfig(tableConfig, schema);
+    } catch (Exception e) {
+      Assert.assertEquals(e.getMessage(), "INCREMENT merger cannot be applied to PK.");
+    }
+
+    partialUpsertStratgies = new HashMap<>();
+    partialUpsertStratgies.put("myCol2", UpsertConfig.Strategy.INCREMENT);
+    try {
+      TableConfigUtils.validateUpsertConfig(tableConfig, schema);
+    } catch (Exception e) {
+      Assert.assertEquals(e.getMessage(), "INCREMENT merger should be numeric data types.");

Review comment:
       I don't think it makes much sense to have an increment time column:
   1. Main time column cannot be changed because it is used to manage the upsert events
   2. Time column should follow the configured time format
   3. I don't see a use case where user would put the delta value for time instead of the actual value. It is hard to manage a flow to have the initial value as the first event




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

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



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


[GitHub] [incubator-pinot] yupeng9 commented on a change in pull request #6899: Add partial upsert config and mergers

Posted by GitBox <gi...@apache.org>.
yupeng9 commented on a change in pull request #6899:
URL: https://github.com/apache/incubator-pinot/pull/6899#discussion_r655555631



##########
File path: pinot-core/src/test/java/org/apache/pinot/core/util/TableConfigUtilsTest.java
##########
@@ -1098,4 +1105,46 @@ public void testValidateUpsertConfig() {
       Assert.assertEquals(e.getMessage(), "The upsert table cannot have star-tree index.");
     }
   }
+
+  @Test
+  public void testValidatePartialUpsertConfig() {
+    Schema schema =
+        new Schema.SchemaBuilder().setSchemaName(TABLE_NAME).addSingleValueDimension("myCol1", FieldSpec.DataType.LONG)
+            .addSingleValueDimension("myCol2", FieldSpec.DataType.STRING)
+            .setPrimaryKeyColumns(Lists.newArrayList("myCol1")).build();
+
+    Map<String, String> streamConfigs = new HashMap<>();
+    streamConfigs.put("stream.kafka.consumer.type", "highLevel");
+    streamConfigs.put("streamType", "kafka");
+    streamConfigs.put("stream.kafka.topic.name", "test");
+    streamConfigs
+        .put("stream.kafka.decoder.class.name", "org.apache.pinot.plugin.stream.kafka.KafkaJSONMessageDecoder");
+    streamConfigs.put("stream.kafka.consumer.type", "simple");
+
+    TableConfig tableConfig = new TableConfigBuilder(TableType.REALTIME).setTableName(TABLE_NAME)
+        .setUpsertConfig(new UpsertConfig(UpsertConfig.Mode.PARTIAL)).setNullHandlingEnabled(false)
+        .setRoutingConfig(new RoutingConfig(null, null, RoutingConfig.STRICT_REPLICA_GROUP_INSTANCE_SELECTOR_TYPE))
+        .setStreamConfigs(streamConfigs).build();
+    try {
+      TableConfigUtils.validateUpsertConfig(tableConfig, schema);
+    } catch (Exception e) {
+      Assert.assertEquals(e.getMessage(), "NullValueHandling is required to be enabled for partial upsert tables.");
+    }
+
+    Map<String, UpsertConfig.Strategy> partialUpsertStratgies = new HashMap<>();
+    partialUpsertStratgies.put("myCol1", UpsertConfig.Strategy.INCREMENT);
+    try {
+      TableConfigUtils.validateUpsertConfig(tableConfig, schema);
+    } catch (Exception e) {
+      Assert.assertEquals(e.getMessage(), "INCREMENT merger cannot be applied to PK.");
+    }
+
+    partialUpsertStratgies = new HashMap<>();
+    partialUpsertStratgies.put("myCol2", UpsertConfig.Strategy.INCREMENT);
+    try {
+      TableConfigUtils.validateUpsertConfig(tableConfig, schema);
+    } catch (Exception e) {
+      Assert.assertEquals(e.getMessage(), "INCREMENT merger should be numeric data types.");

Review comment:
       hmm, one example (contrived) is the driver/rider wait time, which can increment periodically (e.g. +5 min wait time)

##########
File path: pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java
##########
@@ -470,28 +466,32 @@ public void addExtraColumns(Schema newSchema) {
   @Override
   public boolean index(GenericRow row, @Nullable RowMetadata rowMetadata)
       throws IOException {
-    // Update dictionary first
-    updateDictionary(row);
-
-    // If metrics aggregation is enabled and if the dimension values were already seen, this will return existing docId,
-    // else this will return a new docId.
-    int docId = getOrCreateDocId();
-
     boolean canTakeMore;
-    if (docId == _numDocsIndexed) {
-      // New row
+    if (isUpsertEnabled()) {
+      row = handleUpsert(row, _numDocsIndexed);
+
+      updateDictionary(row);
       addNewRow(row);
       // Update number of documents indexed at last to make the latest row queryable
       canTakeMore = _numDocsIndexed++ < _capacity;
-
-      if (isUpsertEnabled()) {
-        handleUpsert(row, docId);
-      }
     } else {
-      Preconditions.checkArgument(!isUpsertEnabled(), "metrics aggregation cannot be used with upsert");
-      assert _aggregateMetrics;
-      aggregateMetrics(row, docId);
-      canTakeMore = true;
+      // Update dictionary first
+      updateDictionary(row);
+
+      // If metrics aggregation is enabled and if the dimension values were already seen, this will return existing
+      // docId, else this will return a new docId.
+      int docId = getOrCreateDocId();
+
+      if (docId == _numDocsIndexed) {
+        // New row
+        addNewRow(row);
+        // Update number of documents indexed at last to make the latest row queryable
+        canTakeMore = _numDocsIndexed++ < _capacity;
+      } else {
+        assert _aggregateMetrics;

Review comment:
       nit: add some error messages for better debugging.

##########
File path: pinot-tools/src/main/java/org/apache/pinot/tools/PartialUpsertQuickStart.java
##########
@@ -0,0 +1,113 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.tools;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import java.io.File;
+import java.net.URL;
+import org.apache.commons.io.FileUtils;
+import org.apache.pinot.common.utils.ZkStarter;
+import org.apache.pinot.spi.plugin.PluginManager;
+import org.apache.pinot.spi.stream.StreamDataProvider;
+import org.apache.pinot.spi.stream.StreamDataServerStartable;
+import org.apache.pinot.tools.Quickstart.Color;
+import org.apache.pinot.tools.admin.command.QuickstartRunner;
+import org.apache.pinot.tools.streams.MeetupRsvpStream;
+import org.apache.pinot.tools.utils.KafkaStarterUtils;
+
+import static org.apache.pinot.tools.Quickstart.prettyPrintResponse;
+import static org.apache.pinot.tools.Quickstart.printStatus;
+
+
+public class PartialUpsertQuickStart {
+  private StreamDataServerStartable _kafkaStarter;
+
+  public static void main(String[] args)
+      throws Exception {
+    PluginManager.get().init();
+    new PartialUpsertQuickStart().execute();
+  }
+
+  // Todo: add a quick start demo
+  public void execute()
+      throws Exception {
+    File quickstartTmpDir = new File(FileUtils.getTempDirectory(), String.valueOf(System.currentTimeMillis()));
+    File bootstrapTableDir = new File(quickstartTmpDir, "meetupRsvp");
+    File dataDir = new File(bootstrapTableDir, "data");
+    Preconditions.checkState(dataDir.mkdirs());
+
+    File schemaFile = new File(bootstrapTableDir, "meetupRsvp_schema.json");
+    File tableConfigFile = new File(bootstrapTableDir, "meetupRsvp_realtime_table_config.json");
+
+    ClassLoader classLoader = Quickstart.class.getClassLoader();
+    URL resource = classLoader.getResource("examples/stream/meetupRsvp/upsert_meetupRsvp_schema.json");
+    Preconditions.checkNotNull(resource);
+    FileUtils.copyURLToFile(resource, schemaFile);
+    resource =
+        classLoader.getResource("examples/stream/meetupRsvp/upsert_partial_meetupRsvp_realtime_table_config.json");
+    Preconditions.checkNotNull(resource);
+    FileUtils.copyURLToFile(resource, tableConfigFile);
+
+    QuickstartTableRequest request = new QuickstartTableRequest(bootstrapTableDir.getAbsolutePath());
+    final QuickstartRunner runner = new QuickstartRunner(Lists.newArrayList(request), 1, 1, 1, dataDir);
+
+    printStatus(Color.CYAN, "***** Starting Kafka *****");
+    final ZkStarter.ZookeeperInstance zookeeperInstance = ZkStarter.startLocalZkServer();
+    try {
+      _kafkaStarter = StreamDataProvider.getServerDataStartable(KafkaStarterUtils.KAFKA_SERVER_STARTABLE_CLASS_NAME,
+          KafkaStarterUtils.getDefaultKafkaConfiguration());
+    } catch (Exception e) {
+      throw new RuntimeException("Failed to start " + KafkaStarterUtils.KAFKA_SERVER_STARTABLE_CLASS_NAME, e);
+    }
+    _kafkaStarter.start();
+    _kafkaStarter.createTopic("meetupRSVPEvents", KafkaStarterUtils.getTopicCreationProps(2));
+    printStatus(Color.CYAN, "***** Starting  meetup data stream and publishing to Kafka *****");
+    MeetupRsvpStream meetupRSVPProvider = new MeetupRsvpStream(true);
+    meetupRSVPProvider.run();
+    printStatus(Color.CYAN, "***** Starting Zookeeper, controller, server and broker *****");
+    runner.startAll();
+    Runtime.getRuntime().addShutdownHook(new Thread(() -> {
+      try {
+        printStatus(Color.GREEN, "***** Shutting down realtime quick start *****");
+        runner.stop();
+        meetupRSVPProvider.stopPublishing();
+        _kafkaStarter.stop();
+        ZkStarter.stopLocalZkServer(zookeeperInstance);
+        FileUtils.deleteDirectory(quickstartTmpDir);
+      } catch (Exception e) {
+        e.printStackTrace();
+      }
+    }));
+    printStatus(Color.CYAN, "***** Bootstrap meetupRSVP(upsert) table *****");
+    runner.bootstrapTable();
+    printStatus(Color.CYAN, "***** Waiting for 15 seconds for a few events to get populated *****");
+    Thread.sleep(15000);
+
+    printStatus(Color.YELLOW, "***** Upsert quickstart setup complete *****");
+
+    String q1 = "select event_id, count(*), sum(rsvp_count) from meetupRsvp group by event_id order by sum(rsvp_count) desc limit 10";
+    printStatus(Color.YELLOW, "Total number of documents, total number of rsvp_counts per event_id in the table");

Review comment:
       add a comment on what the partial upsert behavior that you expect

##########
File path: pinot-segment-local/src/main/java/org/apache/pinot/segment/local/upsert/PartialUpsertHandler.java
##########
@@ -0,0 +1,141 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.segment.local.upsert;
+
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.helix.HelixDataAccessor;
+import org.apache.helix.HelixManager;
+import org.apache.helix.PropertyKey;
+import org.apache.helix.model.CurrentState;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.LiveInstance;
+import org.apache.pinot.segment.local.upsert.merger.PartialUpsertMerger;
+import org.apache.pinot.segment.local.upsert.merger.PartialUpsertMergerFactory;
+import org.apache.pinot.spi.config.table.UpsertConfig;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.utils.CommonConstants.Helix.StateModel.SegmentStateModel;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Handler for partial-upsert.
+ */
+public class PartialUpsertHandler {
+  private static final Logger LOGGER = LoggerFactory.getLogger(PartialUpsertHandler.class);
+
+  private final Map<String, PartialUpsertMerger> _mergers = new HashMap<>();

Review comment:
       +1

##########
File path: pinot-core/src/test/java/org/apache/pinot/core/util/TableConfigUtilsTest.java
##########
@@ -1098,4 +1105,46 @@ public void testValidateUpsertConfig() {
       Assert.assertEquals(e.getMessage(), "The upsert table cannot have star-tree index.");
     }
   }
+
+  @Test
+  public void testValidatePartialUpsertConfig() {
+    Schema schema =
+        new Schema.SchemaBuilder().setSchemaName(TABLE_NAME).addSingleValueDimension("myCol1", FieldSpec.DataType.LONG)
+            .addSingleValueDimension("myCol2", FieldSpec.DataType.STRING)
+            .setPrimaryKeyColumns(Lists.newArrayList("myCol1")).build();
+
+    Map<String, String> streamConfigs = new HashMap<>();
+    streamConfigs.put("stream.kafka.consumer.type", "highLevel");
+    streamConfigs.put("streamType", "kafka");
+    streamConfigs.put("stream.kafka.topic.name", "test");
+    streamConfigs
+        .put("stream.kafka.decoder.class.name", "org.apache.pinot.plugin.stream.kafka.KafkaJSONMessageDecoder");
+    streamConfigs.put("stream.kafka.consumer.type", "simple");
+
+    TableConfig tableConfig = new TableConfigBuilder(TableType.REALTIME).setTableName(TABLE_NAME)
+        .setUpsertConfig(new UpsertConfig(UpsertConfig.Mode.PARTIAL)).setNullHandlingEnabled(false)
+        .setRoutingConfig(new RoutingConfig(null, null, RoutingConfig.STRICT_REPLICA_GROUP_INSTANCE_SELECTOR_TYPE))
+        .setStreamConfigs(streamConfigs).build();
+    try {
+      TableConfigUtils.validateUpsertConfig(tableConfig, schema);
+    } catch (Exception e) {
+      Assert.assertEquals(e.getMessage(), "NullValueHandling is required to be enabled for partial upsert tables.");
+    }
+
+    Map<String, UpsertConfig.Strategy> partialUpsertStratgies = new HashMap<>();
+    partialUpsertStratgies.put("myCol1", UpsertConfig.Strategy.INCREMENT);
+    try {
+      TableConfigUtils.validateUpsertConfig(tableConfig, schema);
+    } catch (Exception e) {
+      Assert.assertEquals(e.getMessage(), "INCREMENT merger cannot be applied to PK.");
+    }
+
+    partialUpsertStratgies = new HashMap<>();
+    partialUpsertStratgies.put("myCol2", UpsertConfig.Strategy.INCREMENT);
+    try {
+      TableConfigUtils.validateUpsertConfig(tableConfig, schema);
+    } catch (Exception e) {
+      Assert.assertEquals(e.getMessage(), "INCREMENT merger should be numeric data types.");

Review comment:
       I see. By time column, I meant a dimension column of `Timestamp` type, which is added recently? Will the check fail this type?

##########
File path: pinot-core/src/test/java/org/apache/pinot/core/util/TableConfigUtilsTest.java
##########
@@ -1098,4 +1105,46 @@ public void testValidateUpsertConfig() {
       Assert.assertEquals(e.getMessage(), "The upsert table cannot have star-tree index.");
     }
   }
+
+  @Test
+  public void testValidatePartialUpsertConfig() {
+    Schema schema =
+        new Schema.SchemaBuilder().setSchemaName(TABLE_NAME).addSingleValueDimension("myCol1", FieldSpec.DataType.LONG)
+            .addSingleValueDimension("myCol2", FieldSpec.DataType.STRING)
+            .setPrimaryKeyColumns(Lists.newArrayList("myCol1")).build();
+
+    Map<String, String> streamConfigs = new HashMap<>();
+    streamConfigs.put("stream.kafka.consumer.type", "highLevel");
+    streamConfigs.put("streamType", "kafka");
+    streamConfigs.put("stream.kafka.topic.name", "test");
+    streamConfigs
+        .put("stream.kafka.decoder.class.name", "org.apache.pinot.plugin.stream.kafka.KafkaJSONMessageDecoder");
+    streamConfigs.put("stream.kafka.consumer.type", "simple");
+
+    TableConfig tableConfig = new TableConfigBuilder(TableType.REALTIME).setTableName(TABLE_NAME)
+        .setUpsertConfig(new UpsertConfig(UpsertConfig.Mode.PARTIAL)).setNullHandlingEnabled(false)
+        .setRoutingConfig(new RoutingConfig(null, null, RoutingConfig.STRICT_REPLICA_GROUP_INSTANCE_SELECTOR_TYPE))
+        .setStreamConfigs(streamConfigs).build();
+    try {
+      TableConfigUtils.validateUpsertConfig(tableConfig, schema);
+    } catch (Exception e) {
+      Assert.assertEquals(e.getMessage(), "NullValueHandling is required to be enabled for partial upsert tables.");
+    }
+
+    Map<String, UpsertConfig.Strategy> partialUpsertStratgies = new HashMap<>();
+    partialUpsertStratgies.put("myCol1", UpsertConfig.Strategy.INCREMENT);
+    try {
+      TableConfigUtils.validateUpsertConfig(tableConfig, schema);
+    } catch (Exception e) {
+      Assert.assertEquals(e.getMessage(), "INCREMENT merger cannot be applied to PK.");
+    }
+
+    partialUpsertStratgies = new HashMap<>();
+    partialUpsertStratgies.put("myCol2", UpsertConfig.Strategy.INCREMENT);
+    try {
+      TableConfigUtils.validateUpsertConfig(tableConfig, schema);
+    } catch (Exception e) {
+      Assert.assertEquals(e.getMessage(), "INCREMENT merger should be numeric data types.");

Review comment:
       I see. So in this case, it's better to model it as a long type?

##########
File path: pinot-core/src/test/java/org/apache/pinot/core/util/TableConfigUtilsTest.java
##########
@@ -1098,4 +1105,46 @@ public void testValidateUpsertConfig() {
       Assert.assertEquals(e.getMessage(), "The upsert table cannot have star-tree index.");
     }
   }
+
+  @Test
+  public void testValidatePartialUpsertConfig() {
+    Schema schema =
+        new Schema.SchemaBuilder().setSchemaName(TABLE_NAME).addSingleValueDimension("myCol1", FieldSpec.DataType.LONG)
+            .addSingleValueDimension("myCol2", FieldSpec.DataType.STRING)
+            .setPrimaryKeyColumns(Lists.newArrayList("myCol1")).build();
+
+    Map<String, String> streamConfigs = new HashMap<>();
+    streamConfigs.put("stream.kafka.consumer.type", "highLevel");
+    streamConfigs.put("streamType", "kafka");
+    streamConfigs.put("stream.kafka.topic.name", "test");
+    streamConfigs
+        .put("stream.kafka.decoder.class.name", "org.apache.pinot.plugin.stream.kafka.KafkaJSONMessageDecoder");
+    streamConfigs.put("stream.kafka.consumer.type", "simple");
+
+    TableConfig tableConfig = new TableConfigBuilder(TableType.REALTIME).setTableName(TABLE_NAME)
+        .setUpsertConfig(new UpsertConfig(UpsertConfig.Mode.PARTIAL)).setNullHandlingEnabled(false)
+        .setRoutingConfig(new RoutingConfig(null, null, RoutingConfig.STRICT_REPLICA_GROUP_INSTANCE_SELECTOR_TYPE))
+        .setStreamConfigs(streamConfigs).build();
+    try {
+      TableConfigUtils.validateUpsertConfig(tableConfig, schema);
+    } catch (Exception e) {
+      Assert.assertEquals(e.getMessage(), "NullValueHandling is required to be enabled for partial upsert tables.");
+    }
+
+    Map<String, UpsertConfig.Strategy> partialUpsertStratgies = new HashMap<>();
+    partialUpsertStratgies.put("myCol1", UpsertConfig.Strategy.INCREMENT);
+    try {
+      TableConfigUtils.validateUpsertConfig(tableConfig, schema);
+    } catch (Exception e) {
+      Assert.assertEquals(e.getMessage(), "INCREMENT merger cannot be applied to PK.");
+    }
+
+    partialUpsertStratgies = new HashMap<>();
+    partialUpsertStratgies.put("myCol2", UpsertConfig.Strategy.INCREMENT);
+    try {
+      TableConfigUtils.validateUpsertConfig(tableConfig, schema);
+    } catch (Exception e) {
+      Assert.assertEquals(e.getMessage(), "INCREMENT merger should be numeric data types.");

Review comment:
       SGTM




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

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



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


[GitHub] [incubator-pinot] deemoliu commented on a change in pull request #6899: Add partial upsert config and mergers

Posted by GitBox <gi...@apache.org>.
deemoliu commented on a change in pull request #6899:
URL: https://github.com/apache/incubator-pinot/pull/6899#discussion_r655708998



##########
File path: pinot-segment-local/src/main/java/org/apache/pinot/segment/local/upsert/PartialUpsertHandler.java
##########
@@ -0,0 +1,141 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.segment.local.upsert;
+
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.helix.HelixDataAccessor;
+import org.apache.helix.HelixManager;
+import org.apache.helix.PropertyKey;
+import org.apache.helix.model.CurrentState;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.LiveInstance;
+import org.apache.pinot.segment.local.upsert.merger.PartialUpsertMerger;
+import org.apache.pinot.segment.local.upsert.merger.PartialUpsertMergerFactory;
+import org.apache.pinot.spi.config.table.UpsertConfig;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.utils.CommonConstants.Helix.StateModel.SegmentStateModel;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Handler for partial-upsert.
+ */
+public class PartialUpsertHandler {
+  private static final Logger LOGGER = LoggerFactory.getLogger(PartialUpsertHandler.class);
+
+  private final Map<String, PartialUpsertMerger> _mergers = new HashMap<>();

Review comment:
       gotcha, thanks for reviewing, will refactor this part.

##########
File path: pinot-tools/src/main/java/org/apache/pinot/tools/PartialUpsertQuickStart.java
##########
@@ -0,0 +1,113 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.tools;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import java.io.File;
+import java.net.URL;
+import org.apache.commons.io.FileUtils;
+import org.apache.pinot.common.utils.ZkStarter;
+import org.apache.pinot.spi.plugin.PluginManager;
+import org.apache.pinot.spi.stream.StreamDataProvider;
+import org.apache.pinot.spi.stream.StreamDataServerStartable;
+import org.apache.pinot.tools.Quickstart.Color;
+import org.apache.pinot.tools.admin.command.QuickstartRunner;
+import org.apache.pinot.tools.streams.MeetupRsvpStream;
+import org.apache.pinot.tools.utils.KafkaStarterUtils;
+
+import static org.apache.pinot.tools.Quickstart.prettyPrintResponse;
+import static org.apache.pinot.tools.Quickstart.printStatus;
+
+
+public class PartialUpsertQuickStart {
+  private StreamDataServerStartable _kafkaStarter;
+
+  public static void main(String[] args)
+      throws Exception {
+    PluginManager.get().init();
+    new PartialUpsertQuickStart().execute();
+  }
+
+  // Todo: add a quick start demo
+  public void execute()
+      throws Exception {
+    File quickstartTmpDir = new File(FileUtils.getTempDirectory(), String.valueOf(System.currentTimeMillis()));
+    File bootstrapTableDir = new File(quickstartTmpDir, "meetupRsvp");
+    File dataDir = new File(bootstrapTableDir, "data");
+    Preconditions.checkState(dataDir.mkdirs());
+
+    File schemaFile = new File(bootstrapTableDir, "meetupRsvp_schema.json");
+    File tableConfigFile = new File(bootstrapTableDir, "meetupRsvp_realtime_table_config.json");
+
+    ClassLoader classLoader = Quickstart.class.getClassLoader();
+    URL resource = classLoader.getResource("examples/stream/meetupRsvp/upsert_meetupRsvp_schema.json");
+    Preconditions.checkNotNull(resource);
+    FileUtils.copyURLToFile(resource, schemaFile);
+    resource =
+        classLoader.getResource("examples/stream/meetupRsvp/upsert_partial_meetupRsvp_realtime_table_config.json");
+    Preconditions.checkNotNull(resource);
+    FileUtils.copyURLToFile(resource, tableConfigFile);
+
+    QuickstartTableRequest request = new QuickstartTableRequest(bootstrapTableDir.getAbsolutePath());
+    final QuickstartRunner runner = new QuickstartRunner(Lists.newArrayList(request), 1, 1, 1, dataDir);
+
+    printStatus(Color.CYAN, "***** Starting Kafka *****");
+    final ZkStarter.ZookeeperInstance zookeeperInstance = ZkStarter.startLocalZkServer();
+    try {
+      _kafkaStarter = StreamDataProvider.getServerDataStartable(KafkaStarterUtils.KAFKA_SERVER_STARTABLE_CLASS_NAME,
+          KafkaStarterUtils.getDefaultKafkaConfiguration());
+    } catch (Exception e) {
+      throw new RuntimeException("Failed to start " + KafkaStarterUtils.KAFKA_SERVER_STARTABLE_CLASS_NAME, e);
+    }
+    _kafkaStarter.start();
+    _kafkaStarter.createTopic("meetupRSVPEvents", KafkaStarterUtils.getTopicCreationProps(2));
+    printStatus(Color.CYAN, "***** Starting  meetup data stream and publishing to Kafka *****");
+    MeetupRsvpStream meetupRSVPProvider = new MeetupRsvpStream(true);
+    meetupRSVPProvider.run();
+    printStatus(Color.CYAN, "***** Starting Zookeeper, controller, server and broker *****");
+    runner.startAll();
+    Runtime.getRuntime().addShutdownHook(new Thread(() -> {
+      try {
+        printStatus(Color.GREEN, "***** Shutting down realtime quick start *****");
+        runner.stop();
+        meetupRSVPProvider.stopPublishing();
+        _kafkaStarter.stop();
+        ZkStarter.stopLocalZkServer(zookeeperInstance);
+        FileUtils.deleteDirectory(quickstartTmpDir);
+      } catch (Exception e) {
+        e.printStackTrace();
+      }
+    }));
+    printStatus(Color.CYAN, "***** Bootstrap meetupRSVP(upsert) table *****");
+    runner.bootstrapTable();
+    printStatus(Color.CYAN, "***** Waiting for 15 seconds for a few events to get populated *****");
+    Thread.sleep(15000);
+
+    printStatus(Color.YELLOW, "***** Upsert quickstart setup complete *****");
+
+    String q1 = "select event_id, count(*), sum(rsvp_count) from meetupRsvp group by event_id order by sum(rsvp_count) desc limit 10";
+    printStatus(Color.YELLOW, "Total number of documents, total number of rsvp_counts per event_id in the table");

Review comment:
       gotcha, added comments.

##########
File path: pinot-segment-local/src/main/java/org/apache/pinot/segment/local/upsert/PartitionUpsertMetadataManager.java
##########
@@ -60,128 +63,170 @@
   private final String _tableNameWithType;
   private final int _partitionId;
   private final ServerMetrics _serverMetrics;
+  private final PartialUpsertHandler _partialUpsertHandler;
 
-  public PartitionUpsertMetadataManager(String tableNameWithType, int partitionId, ServerMetrics serverMetrics) {
+  // TODO(upset): consider an off-heap KV store to persist this index to improve the recovery speed.

Review comment:
       the index here means PK to recordLocation mapping. cc'ed @Jackie-Jiang i re-phrased this comment a bit. 

##########
File path: pinot-segment-local/src/main/java/org/apache/pinot/segment/local/upsert/PartitionUpsertMetadataManager.java
##########
@@ -60,128 +63,170 @@
   private final String _tableNameWithType;
   private final int _partitionId;
   private final ServerMetrics _serverMetrics;
+  private final PartialUpsertHandler _partialUpsertHandler;
 
-  public PartitionUpsertMetadataManager(String tableNameWithType, int partitionId, ServerMetrics serverMetrics) {
+  // TODO(upset): consider an off-heap KV store to persist this index to improve the recovery speed.
+  @VisibleForTesting
+  final ConcurrentHashMap<PrimaryKey, RecordLocation> _primaryKeyToRecordLocationMap = new ConcurrentHashMap<>();
+
+  // Reused for reading previous record during partial upsert
+  private final GenericRow _reuse = new GenericRow();
+  // Stores the result of updateRecord()
+  private GenericRow _result;
+
+  public PartitionUpsertMetadataManager(String tableNameWithType, int partitionId, ServerMetrics serverMetrics,
+      @Nullable PartialUpsertHandler partialUpsertHandler) {
     _tableNameWithType = tableNameWithType;
     _partitionId = partitionId;
     _serverMetrics = serverMetrics;
+    _partialUpsertHandler = partialUpsertHandler;
   }
 
-  public ConcurrentHashMap<PrimaryKey, RecordLocation> getPrimaryKeyToRecordLocationMap() {
-    return _primaryKeyToRecordLocationMap;
-  }
-
-  // TODO(upset): consider an off-heap KV store to persist this index to improve the recovery speed.
-  @VisibleForTesting
-  final ConcurrentHashMap<PrimaryKey, RecordLocation> _primaryKeyToRecordLocationMap = new ConcurrentHashMap<>();
-
   /**
-   * Initializes the upsert metadata for the given immutable segment, returns the valid doc ids for the segment.
+   * Initializes the upsert metadata for the given immutable segment.
    */
-  public ThreadSafeMutableRoaringBitmap addSegment(String segmentName, Iterator<RecordInfo> recordInfoIterator) {
+  public void addSegment(IndexSegment segment, Iterator<RecordInfo> recordInfoIterator) {
+    String segmentName = segment.getSegmentName();
     LOGGER.info("Adding upsert metadata for segment: {}", segmentName);
+    ThreadSafeMutableRoaringBitmap validDocIds = segment.getValidDocIds();
+    assert validDocIds != null;
 
-    ThreadSafeMutableRoaringBitmap validDocIds = new ThreadSafeMutableRoaringBitmap();
     while (recordInfoIterator.hasNext()) {
       RecordInfo recordInfo = recordInfoIterator.next();
       _primaryKeyToRecordLocationMap.compute(recordInfo._primaryKey, (primaryKey, currentRecordLocation) -> {
         if (currentRecordLocation != null) {
           // Existing primary key
 
-          if (segmentName.equals(currentRecordLocation.getSegmentName())) {
-            // The current record location has the same segment name
-
-            // Update the record location when the new timestamp is greater than or equal to the current timestamp.
-            // There are 2 scenarios:
-            //   1. The current record location is pointing to the same segment (the segment being added). In this case,
-            //      we want to update the record location when there is a tie to keep the newer record. Note that the
-            //      record info iterator will return records with incremental doc ids.
-            //   2. The current record location is pointing to the old segment being replaced. This could happen when
-            //      committing a consuming segment, or reloading a completed segment. In this case, we want to update
-            //      the record location when there is a tie because the record locations should point to the new added
-            //      segment instead of the old segment being replaced. Also, do not update the valid doc ids for the old
-            //      segment because it has not been replaced yet.
+          // The current record is in the same segment
+          // Update the record location when there is a tie to keep the newer record. Note that the record info iterator
+          // will return records with incremental doc ids.
+          IndexSegment currentSegment = currentRecordLocation.getSegment();
+          if (segment == currentSegment) {
             if (recordInfo._timestamp >= currentRecordLocation.getTimestamp()) {
-              // Only update the valid doc ids for the new segment
-              if (validDocIds == currentRecordLocation.getValidDocIds()) {
-                validDocIds.remove(currentRecordLocation.getDocId());
-              }
+              validDocIds.remove(currentRecordLocation.getDocId());
               validDocIds.add(recordInfo._docId);
-              return new RecordLocation(segmentName, recordInfo._docId, recordInfo._timestamp, validDocIds);
+              return new RecordLocation(segment, recordInfo._docId, recordInfo._timestamp);
             } else {
               return currentRecordLocation;
             }
-          } else {
-            // The current record location is pointing to a different segment
-
-            // Update the record location when getting a newer timestamp, or the timestamp is the same as the current
-            // timestamp, but the segment has a larger sequence number (the segment is newer than the current segment).
-            if (recordInfo._timestamp > currentRecordLocation.getTimestamp() || (
-                recordInfo._timestamp == currentRecordLocation.getTimestamp()
-                    && LLCSegmentName.isLowLevelConsumerSegmentName(segmentName)
-                    && LLCSegmentName.isLowLevelConsumerSegmentName(currentRecordLocation.getSegmentName())
-                    && LLCSegmentName.getSequenceNumber(segmentName) > LLCSegmentName
-                    .getSequenceNumber(currentRecordLocation.getSegmentName()))) {
-              currentRecordLocation.getValidDocIds().remove(currentRecordLocation.getDocId());
+          }
+
+          // The current record is in an old segment being replaced

Review comment:
       @chenboat This case happens when reloading a completed segment. The old segment is being replaced.
   cc'ed @Jackie-Jiang 
   

##########
File path: pinot-segment-local/src/main/java/org/apache/pinot/segment/local/upsert/merger/PartialUpsertMerger.java
##########
@@ -0,0 +1,30 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.segment.local.upsert.merger;
+
+public interface PartialUpsertMerger {
+  /**
+   * Handle partial upsert merge.
+   *
+   * @param previousValue the last derived full record during ingestion.
+   * @param currentValue the new consumed record.

Review comment:
       thanks for pointing this out. The javadoc is not correct. I updated the javadoc for this function.

##########
File path: pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java
##########
@@ -470,28 +466,32 @@ public void addExtraColumns(Schema newSchema) {
   @Override
   public boolean index(GenericRow row, @Nullable RowMetadata rowMetadata)
       throws IOException {
-    // Update dictionary first
-    updateDictionary(row);
-
-    // If metrics aggregation is enabled and if the dimension values were already seen, this will return existing docId,
-    // else this will return a new docId.
-    int docId = getOrCreateDocId();
-
     boolean canTakeMore;
-    if (docId == _numDocsIndexed) {
-      // New row
+    if (isUpsertEnabled()) {
+      row = handleUpsert(row, _numDocsIndexed);
+
+      updateDictionary(row);
       addNewRow(row);
       // Update number of documents indexed at last to make the latest row queryable
       canTakeMore = _numDocsIndexed++ < _capacity;
-
-      if (isUpsertEnabled()) {
-        handleUpsert(row, docId);
-      }
     } else {
-      Preconditions.checkArgument(!isUpsertEnabled(), "metrics aggregation cannot be used with upsert");
-      assert _aggregateMetrics;
-      aggregateMetrics(row, docId);
-      canTakeMore = true;
+      // Update dictionary first
+      updateDictionary(row);
+
+      // If metrics aggregation is enabled and if the dimension values were already seen, this will return existing
+      // docId, else this will return a new docId.
+      int docId = getOrCreateDocId();
+
+      if (docId == _numDocsIndexed) {
+        // New row
+        addNewRow(row);
+        // Update number of documents indexed at last to make the latest row queryable
+        canTakeMore = _numDocsIndexed++ < _capacity;
+      } else {
+        assert _aggregateMetrics;

Review comment:
       gotcha, added a preCondition check 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.

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] [incubator-pinot] codecov-commenter edited a comment on pull request #6899: Add partial upsert config and mergers

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #6899:
URL: https://github.com/apache/incubator-pinot/pull/6899#issuecomment-853568285


   # [Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6899?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 [#6899](https://codecov.io/gh/apache/incubator-pinot/pull/6899?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (1b49d4a) into [master](https://codecov.io/gh/apache/incubator-pinot/commit/a1c9b631381a25ddd6d3164d6a9ce337c3939b9f?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (a1c9b63) will **decrease** coverage by `31.42%`.
   > The diff coverage is `32.43%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/incubator-pinot/pull/6899/graphs/tree.svg?width=650&height=150&src=pr&token=4ibza2ugkz&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/incubator-pinot/pull/6899?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@              Coverage Diff              @@
   ##             master    #6899       +/-   ##
   =============================================
   - Coverage     73.38%   41.96%   -31.43%     
   + Complexity       12        7        -5     
   =============================================
     Files          1453     1455        +2     
     Lines         72032    72116       +84     
     Branches      10427    10443       +16     
   =============================================
   - Hits          52863    30265    -22598     
   - Misses        15643    39237    +23594     
   + Partials       3526     2614      -912     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration | `41.96% <32.43%> (-0.17%)` | :arrow_down: |
   | unittests | `?` | |
   
   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/incubator-pinot/pull/6899?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/incubator-pinot/pull/6899/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==) | `0.00% <0.00%> (ø)` | |
   | [...t/segment/local/upsert/merger/IncrementMerger.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC91cHNlcnQvbWVyZ2VyL0luY3JlbWVudE1lcmdlci5qYXZh) | `0.00% <0.00%> (ø)` | |
   | [...rg/apache/pinot/spi/config/table/UpsertConfig.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc3BpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcGkvY29uZmlnL3RhYmxlL1Vwc2VydENvbmZpZy5qYXZh) | `0.00% <0.00%> (-87.50%)` | :arrow_down: |
   | [.../org/apache/pinot/spi/data/readers/GenericRow.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc3BpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcGkvZGF0YS9yZWFkZXJzL0dlbmVyaWNSb3cuamF2YQ==) | `0.00% <0.00%> (-64.64%)` | :arrow_down: |
   | [...he/pinot/segment/local/utils/TableConfigUtils.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC91dGlscy9UYWJsZUNvbmZpZ1V0aWxzLmphdmE=) | `48.22% <14.28%> (-33.65%)` | :arrow_down: |
   | [...t/local/upsert/PartitionUpsertMetadataManager.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC91cHNlcnQvUGFydGl0aW9uVXBzZXJ0TWV0YWRhdGFNYW5hZ2VyLmphdmE=) | `53.08% <30.43%> (-42.16%)` | :arrow_down: |
   | [...ata/manager/realtime/RealtimeTableDataManager.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9kYXRhL21hbmFnZXIvcmVhbHRpbWUvUmVhbHRpbWVUYWJsZURhdGFNYW5hZ2VyLmphdmE=) | `67.47% <61.53%> (-1.30%)` | :arrow_down: |
   | [...local/indexsegment/mutable/MutableSegmentImpl.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9pbmRleHNlZ21lbnQvbXV0YWJsZS9NdXRhYmxlU2VnbWVudEltcGwuamF2YQ==) | `73.48% <87.50%> (-5.38%)` | :arrow_down: |
   | [...gment/local/upsert/TableUpsertMetadataManager.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC91cHNlcnQvVGFibGVVcHNlcnRNZXRhZGF0YU1hbmFnZXIuamF2YQ==) | `100.00% <100.00%> (ø)` | |
   | [...c/main/java/org/apache/pinot/common/tier/Tier.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vdGllci9UaWVyLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | ... and [930 more](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6899?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/incubator-pinot/pull/6899?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 [a1c9b63...1b49d4a](https://codecov.io/gh/apache/incubator-pinot/pull/6899?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.

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] [incubator-pinot] codecov-commenter edited a comment on pull request #6899: Add partial upsert config and mergers

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #6899:
URL: https://github.com/apache/incubator-pinot/pull/6899#issuecomment-853568285


   # [Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6899?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 [#6899](https://codecov.io/gh/apache/incubator-pinot/pull/6899?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (32a5f10) into [master](https://codecov.io/gh/apache/incubator-pinot/commit/a1c9b631381a25ddd6d3164d6a9ce337c3939b9f?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (a1c9b63) will **decrease** coverage by `0.13%`.
   > The diff coverage is `39.09%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/incubator-pinot/pull/6899/graphs/tree.svg?width=650&height=150&src=pr&token=4ibza2ugkz&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/incubator-pinot/pull/6899?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #6899      +/-   ##
   ============================================
   - Coverage     73.38%   73.25%   -0.14%     
     Complexity       12       12              
   ============================================
     Files          1453     1457       +4     
     Lines         72032    72139     +107     
     Branches      10427    10449      +22     
   ============================================
   - Hits          52863    52845      -18     
   - Misses        15643    15755     +112     
   - Partials       3526     3539      +13     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration | `41.89% <27.06%> (-0.23%)` | :arrow_down: |
   | unittests | `65.44% <32.33%> (+0.03%)` | :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://codecov.io/gh/apache/incubator-pinot/pull/6899?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/incubator-pinot/pull/6899/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==) | `0.00% <0.00%> (ø)` | |
   | [...inot/segment/local/upsert/merger/IgnoreMerger.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC91cHNlcnQvbWVyZ2VyL0lnbm9yZU1lcmdlci5qYXZh) | `0.00% <0.00%> (ø)` | |
   | [...t/segment/local/upsert/merger/IncrementMerger.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC91cHNlcnQvbWVyZ2VyL0luY3JlbWVudE1lcmdlci5qYXZh) | `0.00% <0.00%> (ø)` | |
   | [...t/segment/local/upsert/merger/OverwriteMerger.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC91cHNlcnQvbWVyZ2VyL092ZXJ3cml0ZU1lcmdlci5qYXZh) | `0.00% <0.00%> (ø)` | |
   | [...he/pinot/segment/local/utils/TableConfigUtils.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC91dGlscy9UYWJsZUNvbmZpZ1V0aWxzLmphdmE=) | `78.43% <11.11%> (-3.44%)` | :arrow_down: |
   | [...t/local/upsert/PartitionUpsertMetadataManager.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC91cHNlcnQvUGFydGl0aW9uVXBzZXJ0TWV0YWRhdGFNYW5hZ2VyLmphdmE=) | `85.18% <52.17%> (-10.06%)` | :arrow_down: |
   | [...ata/manager/realtime/RealtimeTableDataManager.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9kYXRhL21hbmFnZXIvcmVhbHRpbWUvUmVhbHRpbWVUYWJsZURhdGFNYW5hZ2VyLmphdmE=) | `67.47% <61.53%> (-1.30%)` | :arrow_down: |
   | [...rg/apache/pinot/spi/config/table/UpsertConfig.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc3BpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcGkvY29uZmlnL3RhYmxlL1Vwc2VydENvbmZpZy5qYXZh) | `78.26% <73.33%> (-9.24%)` | :arrow_down: |
   | [...local/indexsegment/mutable/MutableSegmentImpl.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9pbmRleHNlZ21lbnQvbXV0YWJsZS9NdXRhYmxlU2VnbWVudEltcGwuamF2YQ==) | `78.75% <87.50%> (-0.12%)` | :arrow_down: |
   | [...gment/local/upsert/TableUpsertMetadataManager.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC91cHNlcnQvVGFibGVVcHNlcnRNZXRhZGF0YU1hbmFnZXIuamF2YQ==) | `100.00% <100.00%> (ø)` | |
   | ... and [38 more](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6899?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/incubator-pinot/pull/6899?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 [a1c9b63...32a5f10](https://codecov.io/gh/apache/incubator-pinot/pull/6899?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.

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] [incubator-pinot] codecov-commenter edited a comment on pull request #6899: Add partial upsert config and mergers

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #6899:
URL: https://github.com/apache/incubator-pinot/pull/6899#issuecomment-853568285


   # [Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6899?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 [#6899](https://codecov.io/gh/apache/incubator-pinot/pull/6899?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (e49861f) into [master](https://codecov.io/gh/apache/incubator-pinot/commit/282edba94d63ddbaabc8bf2cfc55030bf1c1f30d?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (282edba) will **decrease** coverage by `0.05%`.
   > The diff coverage is `59.45%`.
   
   > :exclamation: Current head e49861f differs from pull request most recent head c54b86f. Consider uploading reports for the commit c54b86f to get more accurate results
   [![Impacted file tree graph](https://codecov.io/gh/apache/incubator-pinot/pull/6899/graphs/tree.svg?width=650&height=150&src=pr&token=4ibza2ugkz&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/incubator-pinot/pull/6899?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #6899      +/-   ##
   ============================================
   - Coverage     73.62%   73.57%   -0.06%     
     Complexity       91       91              
   ============================================
     Files          1482     1486       +4     
     Lines         72923    73071     +148     
     Branches      10493    10526      +33     
   ============================================
   + Hits          53693    53759      +66     
   - Misses        15756    15813      +57     
   - Partials       3474     3499      +25     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration | `41.80% <30.18%> (+0.03%)` | :arrow_up: |
   | unittests | `65.52% <50.45%> (-0.06%)` | :arrow_down: |
   
   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/incubator-pinot/pull/6899?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...ocal/indexsegment/immutable/EmptyIndexSegment.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9pbmRleHNlZ21lbnQvaW1tdXRhYmxlL0VtcHR5SW5kZXhTZWdtZW50LmphdmE=) | `36.84% <ø> (ø)` | |
   | [...ocal/indexsegment/mutable/IntermediateSegment.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9pbmRleHNlZ21lbnQvbXV0YWJsZS9JbnRlcm1lZGlhdGVTZWdtZW50LmphdmE=) | `70.06% <ø> (ø)` | |
   | [...local/realtime/impl/geospatial/MutableH3Index.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9yZWFsdGltZS9pbXBsL2dlb3NwYXRpYWwvTXV0YWJsZUgzSW5kZXguamF2YQ==) | `93.33% <ø> (ø)` | |
   | [...mpl/invertedindex/RealtimeInvertedIndexReader.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9yZWFsdGltZS9pbXBsL2ludmVydGVkaW5kZXgvUmVhbHRpbWVJbnZlcnRlZEluZGV4UmVhZGVyLmphdmE=) | `100.00% <ø> (ø)` | |
   | [...ealtime/impl/nullvalue/MutableNullValueVector.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9yZWFsdGltZS9pbXBsL251bGx2YWx1ZS9NdXRhYmxlTnVsbFZhbHVlVmVjdG9yLmphdmE=) | `100.00% <ø> (ø)` | |
   | [...segment/index/readers/ValidDocIndexReaderImpl.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9zZWdtZW50L2luZGV4L3JlYWRlcnMvVmFsaWREb2NJbmRleFJlYWRlckltcGwuamF2YQ==) | `0.00% <ø> (-100.00%)` | :arrow_down: |
   | [...ava/org/apache/pinot/segment/spi/IndexSegment.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1zcGkvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3Bpbm90L3NlZ21lbnQvc3BpL0luZGV4U2VnbWVudC5qYXZh) | `0.00% <ø> (ø)` | |
   | [...ment/spi/index/ThreadSafeMutableRoaringBitmap.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1zcGkvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3Bpbm90L3NlZ21lbnQvc3BpL2luZGV4L1RocmVhZFNhZmVNdXRhYmxlUm9hcmluZ0JpdG1hcC5qYXZh) | `100.00% <ø> (ø)` | |
   | [...not/segment/local/upsert/PartialUpsertHandler.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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==) | `33.33% <33.33%> (ø)` | |
   | [...ava/org/apache/pinot/core/plan/FilterPlanNode.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9wbGFuL0ZpbHRlclBsYW5Ob2RlLmphdmE=) | `79.16% <40.00%> (ø)` | |
   | ... and [38 more](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6899?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/incubator-pinot/pull/6899?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 [282edba...c54b86f](https://codecov.io/gh/apache/incubator-pinot/pull/6899?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.

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] [incubator-pinot] yupeng9 commented on a change in pull request #6899: Add partial upsert config and mergers

Posted by GitBox <gi...@apache.org>.
yupeng9 commented on a change in pull request #6899:
URL: https://github.com/apache/incubator-pinot/pull/6899#discussion_r655603207



##########
File path: pinot-core/src/test/java/org/apache/pinot/core/util/TableConfigUtilsTest.java
##########
@@ -1098,4 +1105,46 @@ public void testValidateUpsertConfig() {
       Assert.assertEquals(e.getMessage(), "The upsert table cannot have star-tree index.");
     }
   }
+
+  @Test
+  public void testValidatePartialUpsertConfig() {
+    Schema schema =
+        new Schema.SchemaBuilder().setSchemaName(TABLE_NAME).addSingleValueDimension("myCol1", FieldSpec.DataType.LONG)
+            .addSingleValueDimension("myCol2", FieldSpec.DataType.STRING)
+            .setPrimaryKeyColumns(Lists.newArrayList("myCol1")).build();
+
+    Map<String, String> streamConfigs = new HashMap<>();
+    streamConfigs.put("stream.kafka.consumer.type", "highLevel");
+    streamConfigs.put("streamType", "kafka");
+    streamConfigs.put("stream.kafka.topic.name", "test");
+    streamConfigs
+        .put("stream.kafka.decoder.class.name", "org.apache.pinot.plugin.stream.kafka.KafkaJSONMessageDecoder");
+    streamConfigs.put("stream.kafka.consumer.type", "simple");
+
+    TableConfig tableConfig = new TableConfigBuilder(TableType.REALTIME).setTableName(TABLE_NAME)
+        .setUpsertConfig(new UpsertConfig(UpsertConfig.Mode.PARTIAL)).setNullHandlingEnabled(false)
+        .setRoutingConfig(new RoutingConfig(null, null, RoutingConfig.STRICT_REPLICA_GROUP_INSTANCE_SELECTOR_TYPE))
+        .setStreamConfigs(streamConfigs).build();
+    try {
+      TableConfigUtils.validateUpsertConfig(tableConfig, schema);
+    } catch (Exception e) {
+      Assert.assertEquals(e.getMessage(), "NullValueHandling is required to be enabled for partial upsert tables.");
+    }
+
+    Map<String, UpsertConfig.Strategy> partialUpsertStratgies = new HashMap<>();
+    partialUpsertStratgies.put("myCol1", UpsertConfig.Strategy.INCREMENT);
+    try {
+      TableConfigUtils.validateUpsertConfig(tableConfig, schema);
+    } catch (Exception e) {
+      Assert.assertEquals(e.getMessage(), "INCREMENT merger cannot be applied to PK.");
+    }
+
+    partialUpsertStratgies = new HashMap<>();
+    partialUpsertStratgies.put("myCol2", UpsertConfig.Strategy.INCREMENT);
+    try {
+      TableConfigUtils.validateUpsertConfig(tableConfig, schema);
+    } catch (Exception e) {
+      Assert.assertEquals(e.getMessage(), "INCREMENT merger should be numeric data types.");

Review comment:
       SGTM




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

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



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


[GitHub] [incubator-pinot] codecov-commenter edited a comment on pull request #6899: Add partial upsert config and mergers

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #6899:
URL: https://github.com/apache/incubator-pinot/pull/6899#issuecomment-853568285


   # [Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6899?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 [#6899](https://codecov.io/gh/apache/incubator-pinot/pull/6899?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (5e33df4) into [master](https://codecov.io/gh/apache/incubator-pinot/commit/a1c9b631381a25ddd6d3164d6a9ce337c3939b9f?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (a1c9b63) will **decrease** coverage by `0.09%`.
   > The diff coverage is `40.00%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/incubator-pinot/pull/6899/graphs/tree.svg?width=650&height=150&src=pr&token=4ibza2ugkz&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/incubator-pinot/pull/6899?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #6899      +/-   ##
   ============================================
   - Coverage     73.38%   73.29%   -0.10%     
     Complexity       12       12              
   ============================================
     Files          1453     1457       +4     
     Lines         72032    72168     +136     
     Branches      10427    10456      +29     
   ============================================
   + Hits          52863    52893      +30     
   - Misses        15643    15726      +83     
   - Partials       3526     3549      +23     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration | `42.08% <28.57%> (-0.04%)` | :arrow_down: |
   | unittests | `65.30% <33.33%> (-0.11%)` | :arrow_down: |
   
   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/incubator-pinot/pull/6899?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...ocal/indexsegment/immutable/EmptyIndexSegment.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9pbmRleHNlZ21lbnQvaW1tdXRhYmxlL0VtcHR5SW5kZXhTZWdtZW50LmphdmE=) | `36.84% <ø> (ø)` | |
   | [...ocal/indexsegment/mutable/IntermediateSegment.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9pbmRleHNlZ21lbnQvbXV0YWJsZS9JbnRlcm1lZGlhdGVTZWdtZW50LmphdmE=) | `70.06% <ø> (ø)` | |
   | [...local/realtime/impl/geospatial/MutableH3Index.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9yZWFsdGltZS9pbXBsL2dlb3NwYXRpYWwvTXV0YWJsZUgzSW5kZXguamF2YQ==) | `93.33% <ø> (ø)` | |
   | [...mpl/invertedindex/RealtimeInvertedIndexReader.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9yZWFsdGltZS9pbXBsL2ludmVydGVkaW5kZXgvUmVhbHRpbWVJbnZlcnRlZEluZGV4UmVhZGVyLmphdmE=) | `100.00% <ø> (ø)` | |
   | [...ealtime/impl/nullvalue/MutableNullValueVector.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9yZWFsdGltZS9pbXBsL251bGx2YWx1ZS9NdXRhYmxlTnVsbFZhbHVlVmVjdG9yLmphdmE=) | `100.00% <ø> (ø)` | |
   | [...segment/index/readers/ValidDocIndexReaderImpl.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9zZWdtZW50L2luZGV4L3JlYWRlcnMvVmFsaWREb2NJbmRleFJlYWRlckltcGwuamF2YQ==) | `0.00% <ø> (-100.00%)` | :arrow_down: |
   | [...not/segment/local/upsert/PartialUpsertHandler.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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==) | `0.00% <0.00%> (ø)` | |
   | [...t/segment/local/upsert/merger/IncrementMerger.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC91cHNlcnQvbWVyZ2VyL0luY3JlbWVudE1lcmdlci5qYXZh) | `0.00% <0.00%> (ø)` | |
   | [...t/segment/local/upsert/merger/OverwriteMerger.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC91cHNlcnQvbWVyZ2VyL092ZXJ3cml0ZU1lcmdlci5qYXZh) | `0.00% <0.00%> (ø)` | |
   | [...ocal/upsert/merger/PartialUpsertMergerFactory.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC91cHNlcnQvbWVyZ2VyL1BhcnRpYWxVcHNlcnRNZXJnZXJGYWN0b3J5LmphdmE=) | `0.00% <0.00%> (ø)` | |
   | ... and [44 more](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6899?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/incubator-pinot/pull/6899?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 [a1c9b63...5e33df4](https://codecov.io/gh/apache/incubator-pinot/pull/6899?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.

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] [incubator-pinot] codecov-commenter edited a comment on pull request #6899: Add partial upsert config and mergers

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #6899:
URL: https://github.com/apache/incubator-pinot/pull/6899#issuecomment-853568285


   # [Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6899?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 [#6899](https://codecov.io/gh/apache/incubator-pinot/pull/6899?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (ac0fad9) into [master](https://codecov.io/gh/apache/incubator-pinot/commit/a1c9b631381a25ddd6d3164d6a9ce337c3939b9f?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (a1c9b63) will **decrease** coverage by `8.02%`.
   > The diff coverage is `43.75%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/incubator-pinot/pull/6899/graphs/tree.svg?width=650&height=150&src=pr&token=4ibza2ugkz&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/incubator-pinot/pull/6899?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #6899      +/-   ##
   ============================================
   - Coverage     73.38%   65.36%   -8.03%     
     Complexity       12       12              
   ============================================
     Files          1453     1457       +4     
     Lines         72032    72118      +86     
     Branches      10427    10448      +21     
   ============================================
   - Hits          52863    47141    -5722     
   - Misses        15643    21561    +5918     
   + Partials       3526     3416     -110     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration | `?` | |
   | unittests | `65.36% <43.75%> (-0.05%)` | :arrow_down: |
   
   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/incubator-pinot/pull/6899?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...ata/manager/realtime/RealtimeTableDataManager.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9kYXRhL21hbmFnZXIvcmVhbHRpbWUvUmVhbHRpbWVUYWJsZURhdGFNYW5hZ2VyLmphdmE=) | `10.69% <0.00%> (-58.08%)` | :arrow_down: |
   | [...ocal/indexsegment/immutable/EmptyIndexSegment.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9pbmRleHNlZ21lbnQvaW1tdXRhYmxlL0VtcHR5SW5kZXhTZWdtZW50LmphdmE=) | `36.84% <ø> (ø)` | |
   | [...ocal/indexsegment/mutable/IntermediateSegment.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9pbmRleHNlZ21lbnQvbXV0YWJsZS9JbnRlcm1lZGlhdGVTZWdtZW50LmphdmE=) | `70.06% <ø> (ø)` | |
   | [...local/realtime/impl/geospatial/MutableH3Index.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9yZWFsdGltZS9pbXBsL2dlb3NwYXRpYWwvTXV0YWJsZUgzSW5kZXguamF2YQ==) | `93.33% <ø> (ø)` | |
   | [...mpl/invertedindex/RealtimeInvertedIndexReader.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9yZWFsdGltZS9pbXBsL2ludmVydGVkaW5kZXgvUmVhbHRpbWVJbnZlcnRlZEluZGV4UmVhZGVyLmphdmE=) | `100.00% <ø> (ø)` | |
   | [...ealtime/impl/nullvalue/MutableNullValueVector.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9yZWFsdGltZS9pbXBsL251bGx2YWx1ZS9NdXRhYmxlTnVsbFZhbHVlVmVjdG9yLmphdmE=) | `85.71% <ø> (-14.29%)` | :arrow_down: |
   | [...segment/index/readers/ValidDocIndexReaderImpl.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9zZWdtZW50L2luZGV4L3JlYWRlcnMvVmFsaWREb2NJbmRleFJlYWRlckltcGwuamF2YQ==) | `0.00% <ø> (-100.00%)` | :arrow_down: |
   | [...not/segment/local/upsert/PartialUpsertHandler.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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==) | `0.00% <0.00%> (ø)` | |
   | [...t/segment/local/upsert/merger/IncrementMerger.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC91cHNlcnQvbWVyZ2VyL0luY3JlbWVudE1lcmdlci5qYXZh) | `0.00% <0.00%> (ø)` | |
   | [...t/segment/local/upsert/merger/OverwriteMerger.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC91cHNlcnQvbWVyZ2VyL092ZXJ3cml0ZU1lcmdlci5qYXZh) | `0.00% <0.00%> (ø)` | |
   | ... and [371 more](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6899?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/incubator-pinot/pull/6899?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 [a1c9b63...ac0fad9](https://codecov.io/gh/apache/incubator-pinot/pull/6899?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.

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] [incubator-pinot] codecov-commenter edited a comment on pull request #6899: Add partial upsert config and mergers

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #6899:
URL: https://github.com/apache/incubator-pinot/pull/6899#issuecomment-853568285


   # [Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6899?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 [#6899](https://codecov.io/gh/apache/incubator-pinot/pull/6899?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (e3de513) into [master](https://codecov.io/gh/apache/incubator-pinot/commit/0185482d9da2ac299b4b15bcd2998165ccbbdf71?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (0185482) will **decrease** coverage by `7.89%`.
   > The diff coverage is `57.29%`.
   
   > :exclamation: Current head e3de513 differs from pull request most recent head 8098c8b. Consider uploading reports for the commit 8098c8b to get more accurate results
   [![Impacted file tree graph](https://codecov.io/gh/apache/incubator-pinot/pull/6899/graphs/tree.svg?width=650&height=150&src=pr&token=4ibza2ugkz&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/incubator-pinot/pull/6899?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #6899      +/-   ##
   ============================================
   - Coverage     73.23%   65.34%   -7.90%     
     Complexity       12       12              
   ============================================
     Files          1439     1456      +17     
     Lines         71333    72065     +732     
     Branches      10334    10439     +105     
   ============================================
   - Hits          52243    47093    -5150     
   - Misses        15578    21567    +5989     
   + Partials       3512     3405     -107     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration | `?` | |
   | unittests | `65.34% <57.29%> (-0.01%)` | :arrow_down: |
   
   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/incubator-pinot/pull/6899?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...roker/requesthandler/BaseBrokerRequestHandler.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3QtYnJva2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9icm9rZXIvcmVxdWVzdGhhbmRsZXIvQmFzZUJyb2tlclJlcXVlc3RIYW5kbGVyLmphdmE=) | `17.37% <0.00%> (-53.88%)` | :arrow_down: |
   | [...e/pinot/common/minion/MergeRollupTaskMetadata.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWluaW9uL01lcmdlUm9sbHVwVGFza01ldGFkYXRhLmphdmE=) | `0.00% <0.00%> (ø)` | |
   | [...e/pinot/common/minion/MinionTaskMetadataUtils.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWluaW9uL01pbmlvblRhc2tNZXRhZGF0YVV0aWxzLmphdmE=) | `0.00% <0.00%> (-75.00%)` | :arrow_down: |
   | [...mmon/restlet/resources/SegmentServerDebugInfo.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vcmVzdGxldC9yZXNvdXJjZXMvU2VnbWVudFNlcnZlckRlYnVnSW5mby5qYXZh) | `0.00% <0.00%> (ø)` | |
   | [...che/pinot/controller/api/debug/TableDebugInfo.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3QtY29udHJvbGxlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29udHJvbGxlci9hcGkvZGVidWcvVGFibGVEZWJ1Z0luZm8uamF2YQ==) | `0.00% <0.00%> (ø)` | |
   | [...ces/PinotSegmentUploadDownloadRestletResource.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3QtY29udHJvbGxlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29udHJvbGxlci9hcGkvcmVzb3VyY2VzL1Bpbm90U2VnbWVudFVwbG9hZERvd25sb2FkUmVzdGxldFJlc291cmNlLmphdmE=) | `10.00% <0.00%> (-45.03%)` | :arrow_down: |
   | [...t/controller/api/resources/TableDebugResource.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3QtY29udHJvbGxlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29udHJvbGxlci9hcGkvcmVzb3VyY2VzL1RhYmxlRGVidWdSZXNvdXJjZS5qYXZh) | `0.00% <0.00%> (ø)` | |
   | [...troller/helix/core/minion/ClusterInfoAccessor.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3QtY29udHJvbGxlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29udHJvbGxlci9oZWxpeC9jb3JlL21pbmlvbi9DbHVzdGVySW5mb0FjY2Vzc29yLmphdmE=) | `32.14% <0.00%> (-47.86%)` | :arrow_down: |
   | [...x/core/minion/generator/TaskGeneratorRegistry.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3QtY29udHJvbGxlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29udHJvbGxlci9oZWxpeC9jb3JlL21pbmlvbi9nZW5lcmF0b3IvVGFza0dlbmVyYXRvclJlZ2lzdHJ5LmphdmE=) | `76.00% <ø> (-4.00%)` | :arrow_down: |
   | [.../org/apache/pinot/core/common/MinionConstants.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9jb21tb24vTWluaW9uQ29uc3RhbnRzLmphdmE=) | `0.00% <ø> (ø)` | |
   | ... and [430 more](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6899?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/incubator-pinot/pull/6899?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 [0185482...8098c8b](https://codecov.io/gh/apache/incubator-pinot/pull/6899?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.

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] [incubator-pinot] chenboat commented on pull request #6899: Add partial upsert config and mergers (WIP)

Posted by GitBox <gi...@apache.org>.
chenboat commented on pull request #6899:
URL: https://github.com/apache/incubator-pinot/pull/6899#issuecomment-843641824


   Add more description and summary on the work done in this PR. Link to relevant section of the design doc.


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

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



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


[GitHub] [incubator-pinot] codecov-commenter edited a comment on pull request #6899: Add partial upsert config and mergers

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #6899:
URL: https://github.com/apache/incubator-pinot/pull/6899#issuecomment-853568285


   # [Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6899?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 [#6899](https://codecov.io/gh/apache/incubator-pinot/pull/6899?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (aa706ba) into [master](https://codecov.io/gh/apache/incubator-pinot/commit/01f206b1ef69ebaf27f0ac9fc1d92518b85bf0b1?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (01f206b) will **increase** coverage by `0.14%`.
   > The diff coverage is `59.45%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/incubator-pinot/pull/6899/graphs/tree.svg?width=650&height=150&src=pr&token=4ibza2ugkz&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/incubator-pinot/pull/6899?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #6899      +/-   ##
   ============================================
   + Coverage     73.41%   73.55%   +0.14%     
   - Complexity       12       91      +79     
   ============================================
     Files          1454     1486      +32     
     Lines         72127    73071     +944     
     Branches      10448    10526      +78     
   ============================================
   + Hits          52951    53749     +798     
   - Misses        15637    15825     +188     
   + Partials       3539     3497      -42     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration | `41.80% <30.18%> (-0.26%)` | :arrow_down: |
   | unittests | `65.51% <50.45%> (+0.09%)` | :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://codecov.io/gh/apache/incubator-pinot/pull/6899?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...ocal/indexsegment/immutable/EmptyIndexSegment.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9pbmRleHNlZ21lbnQvaW1tdXRhYmxlL0VtcHR5SW5kZXhTZWdtZW50LmphdmE=) | `36.84% <ø> (ø)` | |
   | [...ocal/indexsegment/mutable/IntermediateSegment.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9pbmRleHNlZ21lbnQvbXV0YWJsZS9JbnRlcm1lZGlhdGVTZWdtZW50LmphdmE=) | `70.06% <ø> (ø)` | |
   | [...local/realtime/impl/geospatial/MutableH3Index.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9yZWFsdGltZS9pbXBsL2dlb3NwYXRpYWwvTXV0YWJsZUgzSW5kZXguamF2YQ==) | `93.33% <ø> (ø)` | |
   | [...mpl/invertedindex/RealtimeInvertedIndexReader.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9yZWFsdGltZS9pbXBsL2ludmVydGVkaW5kZXgvUmVhbHRpbWVJbnZlcnRlZEluZGV4UmVhZGVyLmphdmE=) | `100.00% <ø> (ø)` | |
   | [...ealtime/impl/nullvalue/MutableNullValueVector.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9yZWFsdGltZS9pbXBsL251bGx2YWx1ZS9NdXRhYmxlTnVsbFZhbHVlVmVjdG9yLmphdmE=) | `100.00% <ø> (ø)` | |
   | [...segment/index/readers/ValidDocIndexReaderImpl.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9zZWdtZW50L2luZGV4L3JlYWRlcnMvVmFsaWREb2NJbmRleFJlYWRlckltcGwuamF2YQ==) | `0.00% <ø> (-100.00%)` | :arrow_down: |
   | [...ava/org/apache/pinot/segment/spi/IndexSegment.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1zcGkvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3Bpbm90L3NlZ21lbnQvc3BpL0luZGV4U2VnbWVudC5qYXZh) | `0.00% <ø> (ø)` | |
   | [...ment/spi/index/ThreadSafeMutableRoaringBitmap.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1zcGkvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3Bpbm90L3NlZ21lbnQvc3BpL2luZGV4L1RocmVhZFNhZmVNdXRhYmxlUm9hcmluZ0JpdG1hcC5qYXZh) | `100.00% <ø> (ø)` | |
   | [...not/segment/local/upsert/PartialUpsertHandler.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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==) | `33.33% <33.33%> (ø)` | |
   | [...ava/org/apache/pinot/core/plan/FilterPlanNode.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9wbGFuL0ZpbHRlclBsYW5Ob2RlLmphdmE=) | `79.16% <40.00%> (ø)` | |
   | ... and [396 more](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6899?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/incubator-pinot/pull/6899?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 [01f206b...aa706ba](https://codecov.io/gh/apache/incubator-pinot/pull/6899?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.

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] [incubator-pinot] Jackie-Jiang commented on a change in pull request #6899: Add partial upsert config and mergers

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on a change in pull request #6899:
URL: https://github.com/apache/incubator-pinot/pull/6899#discussion_r655597069



##########
File path: pinot-core/src/test/java/org/apache/pinot/core/util/TableConfigUtilsTest.java
##########
@@ -1098,4 +1105,46 @@ public void testValidateUpsertConfig() {
       Assert.assertEquals(e.getMessage(), "The upsert table cannot have star-tree index.");
     }
   }
+
+  @Test
+  public void testValidatePartialUpsertConfig() {
+    Schema schema =
+        new Schema.SchemaBuilder().setSchemaName(TABLE_NAME).addSingleValueDimension("myCol1", FieldSpec.DataType.LONG)
+            .addSingleValueDimension("myCol2", FieldSpec.DataType.STRING)
+            .setPrimaryKeyColumns(Lists.newArrayList("myCol1")).build();
+
+    Map<String, String> streamConfigs = new HashMap<>();
+    streamConfigs.put("stream.kafka.consumer.type", "highLevel");
+    streamConfigs.put("streamType", "kafka");
+    streamConfigs.put("stream.kafka.topic.name", "test");
+    streamConfigs
+        .put("stream.kafka.decoder.class.name", "org.apache.pinot.plugin.stream.kafka.KafkaJSONMessageDecoder");
+    streamConfigs.put("stream.kafka.consumer.type", "simple");
+
+    TableConfig tableConfig = new TableConfigBuilder(TableType.REALTIME).setTableName(TABLE_NAME)
+        .setUpsertConfig(new UpsertConfig(UpsertConfig.Mode.PARTIAL)).setNullHandlingEnabled(false)
+        .setRoutingConfig(new RoutingConfig(null, null, RoutingConfig.STRICT_REPLICA_GROUP_INSTANCE_SELECTOR_TYPE))
+        .setStreamConfigs(streamConfigs).build();
+    try {
+      TableConfigUtils.validateUpsertConfig(tableConfig, schema);
+    } catch (Exception e) {
+      Assert.assertEquals(e.getMessage(), "NullValueHandling is required to be enabled for partial upsert tables.");
+    }
+
+    Map<String, UpsertConfig.Strategy> partialUpsertStratgies = new HashMap<>();
+    partialUpsertStratgies.put("myCol1", UpsertConfig.Strategy.INCREMENT);
+    try {
+      TableConfigUtils.validateUpsertConfig(tableConfig, schema);
+    } catch (Exception e) {
+      Assert.assertEquals(e.getMessage(), "INCREMENT merger cannot be applied to PK.");
+    }
+
+    partialUpsertStratgies = new HashMap<>();
+    partialUpsertStratgies.put("myCol2", UpsertConfig.Strategy.INCREMENT);
+    try {
+      TableConfigUtils.validateUpsertConfig(tableConfig, schema);
+    } catch (Exception e) {
+      Assert.assertEquals(e.getMessage(), "INCREMENT merger should be numeric data types.");

Review comment:
       It won't fail, but still we should not try to increment the timestamp because there is no guarantee that the base value is already ingested (or very hard to manage at least)




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

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



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


[GitHub] [incubator-pinot] yupeng9 commented on a change in pull request #6899: Add partial upsert config and mergers

Posted by GitBox <gi...@apache.org>.
yupeng9 commented on a change in pull request #6899:
URL: https://github.com/apache/incubator-pinot/pull/6899#discussion_r655568342



##########
File path: pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java
##########
@@ -470,28 +466,32 @@ public void addExtraColumns(Schema newSchema) {
   @Override
   public boolean index(GenericRow row, @Nullable RowMetadata rowMetadata)
       throws IOException {
-    // Update dictionary first
-    updateDictionary(row);
-
-    // If metrics aggregation is enabled and if the dimension values were already seen, this will return existing docId,
-    // else this will return a new docId.
-    int docId = getOrCreateDocId();
-
     boolean canTakeMore;
-    if (docId == _numDocsIndexed) {
-      // New row
+    if (isUpsertEnabled()) {
+      row = handleUpsert(row, _numDocsIndexed);
+
+      updateDictionary(row);
       addNewRow(row);
       // Update number of documents indexed at last to make the latest row queryable
       canTakeMore = _numDocsIndexed++ < _capacity;
-
-      if (isUpsertEnabled()) {
-        handleUpsert(row, docId);
-      }
     } else {
-      Preconditions.checkArgument(!isUpsertEnabled(), "metrics aggregation cannot be used with upsert");
-      assert _aggregateMetrics;
-      aggregateMetrics(row, docId);
-      canTakeMore = true;
+      // Update dictionary first
+      updateDictionary(row);
+
+      // If metrics aggregation is enabled and if the dimension values were already seen, this will return existing
+      // docId, else this will return a new docId.
+      int docId = getOrCreateDocId();
+
+      if (docId == _numDocsIndexed) {
+        // New row
+        addNewRow(row);
+        // Update number of documents indexed at last to make the latest row queryable
+        canTakeMore = _numDocsIndexed++ < _capacity;
+      } else {
+        assert _aggregateMetrics;

Review comment:
       nit: add some error messages for better debugging.

##########
File path: pinot-tools/src/main/java/org/apache/pinot/tools/PartialUpsertQuickStart.java
##########
@@ -0,0 +1,113 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.tools;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import java.io.File;
+import java.net.URL;
+import org.apache.commons.io.FileUtils;
+import org.apache.pinot.common.utils.ZkStarter;
+import org.apache.pinot.spi.plugin.PluginManager;
+import org.apache.pinot.spi.stream.StreamDataProvider;
+import org.apache.pinot.spi.stream.StreamDataServerStartable;
+import org.apache.pinot.tools.Quickstart.Color;
+import org.apache.pinot.tools.admin.command.QuickstartRunner;
+import org.apache.pinot.tools.streams.MeetupRsvpStream;
+import org.apache.pinot.tools.utils.KafkaStarterUtils;
+
+import static org.apache.pinot.tools.Quickstart.prettyPrintResponse;
+import static org.apache.pinot.tools.Quickstart.printStatus;
+
+
+public class PartialUpsertQuickStart {
+  private StreamDataServerStartable _kafkaStarter;
+
+  public static void main(String[] args)
+      throws Exception {
+    PluginManager.get().init();
+    new PartialUpsertQuickStart().execute();
+  }
+
+  // Todo: add a quick start demo
+  public void execute()
+      throws Exception {
+    File quickstartTmpDir = new File(FileUtils.getTempDirectory(), String.valueOf(System.currentTimeMillis()));
+    File bootstrapTableDir = new File(quickstartTmpDir, "meetupRsvp");
+    File dataDir = new File(bootstrapTableDir, "data");
+    Preconditions.checkState(dataDir.mkdirs());
+
+    File schemaFile = new File(bootstrapTableDir, "meetupRsvp_schema.json");
+    File tableConfigFile = new File(bootstrapTableDir, "meetupRsvp_realtime_table_config.json");
+
+    ClassLoader classLoader = Quickstart.class.getClassLoader();
+    URL resource = classLoader.getResource("examples/stream/meetupRsvp/upsert_meetupRsvp_schema.json");
+    Preconditions.checkNotNull(resource);
+    FileUtils.copyURLToFile(resource, schemaFile);
+    resource =
+        classLoader.getResource("examples/stream/meetupRsvp/upsert_partial_meetupRsvp_realtime_table_config.json");
+    Preconditions.checkNotNull(resource);
+    FileUtils.copyURLToFile(resource, tableConfigFile);
+
+    QuickstartTableRequest request = new QuickstartTableRequest(bootstrapTableDir.getAbsolutePath());
+    final QuickstartRunner runner = new QuickstartRunner(Lists.newArrayList(request), 1, 1, 1, dataDir);
+
+    printStatus(Color.CYAN, "***** Starting Kafka *****");
+    final ZkStarter.ZookeeperInstance zookeeperInstance = ZkStarter.startLocalZkServer();
+    try {
+      _kafkaStarter = StreamDataProvider.getServerDataStartable(KafkaStarterUtils.KAFKA_SERVER_STARTABLE_CLASS_NAME,
+          KafkaStarterUtils.getDefaultKafkaConfiguration());
+    } catch (Exception e) {
+      throw new RuntimeException("Failed to start " + KafkaStarterUtils.KAFKA_SERVER_STARTABLE_CLASS_NAME, e);
+    }
+    _kafkaStarter.start();
+    _kafkaStarter.createTopic("meetupRSVPEvents", KafkaStarterUtils.getTopicCreationProps(2));
+    printStatus(Color.CYAN, "***** Starting  meetup data stream and publishing to Kafka *****");
+    MeetupRsvpStream meetupRSVPProvider = new MeetupRsvpStream(true);
+    meetupRSVPProvider.run();
+    printStatus(Color.CYAN, "***** Starting Zookeeper, controller, server and broker *****");
+    runner.startAll();
+    Runtime.getRuntime().addShutdownHook(new Thread(() -> {
+      try {
+        printStatus(Color.GREEN, "***** Shutting down realtime quick start *****");
+        runner.stop();
+        meetupRSVPProvider.stopPublishing();
+        _kafkaStarter.stop();
+        ZkStarter.stopLocalZkServer(zookeeperInstance);
+        FileUtils.deleteDirectory(quickstartTmpDir);
+      } catch (Exception e) {
+        e.printStackTrace();
+      }
+    }));
+    printStatus(Color.CYAN, "***** Bootstrap meetupRSVP(upsert) table *****");
+    runner.bootstrapTable();
+    printStatus(Color.CYAN, "***** Waiting for 15 seconds for a few events to get populated *****");
+    Thread.sleep(15000);
+
+    printStatus(Color.YELLOW, "***** Upsert quickstart setup complete *****");
+
+    String q1 = "select event_id, count(*), sum(rsvp_count) from meetupRsvp group by event_id order by sum(rsvp_count) desc limit 10";
+    printStatus(Color.YELLOW, "Total number of documents, total number of rsvp_counts per event_id in the table");

Review comment:
       add a comment on what the partial upsert behavior that you expect

##########
File path: pinot-segment-local/src/main/java/org/apache/pinot/segment/local/upsert/PartialUpsertHandler.java
##########
@@ -0,0 +1,141 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.segment.local.upsert;
+
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.helix.HelixDataAccessor;
+import org.apache.helix.HelixManager;
+import org.apache.helix.PropertyKey;
+import org.apache.helix.model.CurrentState;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.LiveInstance;
+import org.apache.pinot.segment.local.upsert.merger.PartialUpsertMerger;
+import org.apache.pinot.segment.local.upsert.merger.PartialUpsertMergerFactory;
+import org.apache.pinot.spi.config.table.UpsertConfig;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.utils.CommonConstants.Helix.StateModel.SegmentStateModel;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Handler for partial-upsert.
+ */
+public class PartialUpsertHandler {
+  private static final Logger LOGGER = LoggerFactory.getLogger(PartialUpsertHandler.class);
+
+  private final Map<String, PartialUpsertMerger> _mergers = new HashMap<>();

Review comment:
       +1




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

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



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


[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #6899: Add partial upsert config and mergers

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on a change in pull request #6899:
URL: https://github.com/apache/incubator-pinot/pull/6899#discussion_r655586939



##########
File path: pinot-core/src/test/java/org/apache/pinot/core/util/TableConfigUtilsTest.java
##########
@@ -1098,4 +1105,46 @@ public void testValidateUpsertConfig() {
       Assert.assertEquals(e.getMessage(), "The upsert table cannot have star-tree index.");
     }
   }
+
+  @Test
+  public void testValidatePartialUpsertConfig() {
+    Schema schema =
+        new Schema.SchemaBuilder().setSchemaName(TABLE_NAME).addSingleValueDimension("myCol1", FieldSpec.DataType.LONG)
+            .addSingleValueDimension("myCol2", FieldSpec.DataType.STRING)
+            .setPrimaryKeyColumns(Lists.newArrayList("myCol1")).build();
+
+    Map<String, String> streamConfigs = new HashMap<>();
+    streamConfigs.put("stream.kafka.consumer.type", "highLevel");
+    streamConfigs.put("streamType", "kafka");
+    streamConfigs.put("stream.kafka.topic.name", "test");
+    streamConfigs
+        .put("stream.kafka.decoder.class.name", "org.apache.pinot.plugin.stream.kafka.KafkaJSONMessageDecoder");
+    streamConfigs.put("stream.kafka.consumer.type", "simple");
+
+    TableConfig tableConfig = new TableConfigBuilder(TableType.REALTIME).setTableName(TABLE_NAME)
+        .setUpsertConfig(new UpsertConfig(UpsertConfig.Mode.PARTIAL)).setNullHandlingEnabled(false)
+        .setRoutingConfig(new RoutingConfig(null, null, RoutingConfig.STRICT_REPLICA_GROUP_INSTANCE_SELECTOR_TYPE))
+        .setStreamConfigs(streamConfigs).build();
+    try {
+      TableConfigUtils.validateUpsertConfig(tableConfig, schema);
+    } catch (Exception e) {
+      Assert.assertEquals(e.getMessage(), "NullValueHandling is required to be enabled for partial upsert tables.");
+    }
+
+    Map<String, UpsertConfig.Strategy> partialUpsertStratgies = new HashMap<>();
+    partialUpsertStratgies.put("myCol1", UpsertConfig.Strategy.INCREMENT);
+    try {
+      TableConfigUtils.validateUpsertConfig(tableConfig, schema);
+    } catch (Exception e) {
+      Assert.assertEquals(e.getMessage(), "INCREMENT merger cannot be applied to PK.");
+    }
+
+    partialUpsertStratgies = new HashMap<>();
+    partialUpsertStratgies.put("myCol2", UpsertConfig.Strategy.INCREMENT);
+    try {
+      TableConfigUtils.validateUpsertConfig(tableConfig, schema);
+    } catch (Exception e) {
+      Assert.assertEquals(e.getMessage(), "INCREMENT merger should be numeric data types.");

Review comment:
       @yupeng9 The wait time is not really a time column, but a dimension column. Time column must be timestamp or datetime. There is no limit on the data type, as long as the value is stored as numeric type

##########
File path: pinot-core/src/test/java/org/apache/pinot/core/util/TableConfigUtilsTest.java
##########
@@ -1098,4 +1105,46 @@ public void testValidateUpsertConfig() {
       Assert.assertEquals(e.getMessage(), "The upsert table cannot have star-tree index.");
     }
   }
+
+  @Test
+  public void testValidatePartialUpsertConfig() {
+    Schema schema =
+        new Schema.SchemaBuilder().setSchemaName(TABLE_NAME).addSingleValueDimension("myCol1", FieldSpec.DataType.LONG)
+            .addSingleValueDimension("myCol2", FieldSpec.DataType.STRING)
+            .setPrimaryKeyColumns(Lists.newArrayList("myCol1")).build();
+
+    Map<String, String> streamConfigs = new HashMap<>();
+    streamConfigs.put("stream.kafka.consumer.type", "highLevel");
+    streamConfigs.put("streamType", "kafka");
+    streamConfigs.put("stream.kafka.topic.name", "test");
+    streamConfigs
+        .put("stream.kafka.decoder.class.name", "org.apache.pinot.plugin.stream.kafka.KafkaJSONMessageDecoder");
+    streamConfigs.put("stream.kafka.consumer.type", "simple");
+
+    TableConfig tableConfig = new TableConfigBuilder(TableType.REALTIME).setTableName(TABLE_NAME)
+        .setUpsertConfig(new UpsertConfig(UpsertConfig.Mode.PARTIAL)).setNullHandlingEnabled(false)
+        .setRoutingConfig(new RoutingConfig(null, null, RoutingConfig.STRICT_REPLICA_GROUP_INSTANCE_SELECTOR_TYPE))
+        .setStreamConfigs(streamConfigs).build();
+    try {
+      TableConfigUtils.validateUpsertConfig(tableConfig, schema);
+    } catch (Exception e) {
+      Assert.assertEquals(e.getMessage(), "NullValueHandling is required to be enabled for partial upsert tables.");
+    }
+
+    Map<String, UpsertConfig.Strategy> partialUpsertStratgies = new HashMap<>();
+    partialUpsertStratgies.put("myCol1", UpsertConfig.Strategy.INCREMENT);
+    try {
+      TableConfigUtils.validateUpsertConfig(tableConfig, schema);
+    } catch (Exception e) {
+      Assert.assertEquals(e.getMessage(), "INCREMENT merger cannot be applied to PK.");
+    }
+
+    partialUpsertStratgies = new HashMap<>();
+    partialUpsertStratgies.put("myCol2", UpsertConfig.Strategy.INCREMENT);
+    try {
+      TableConfigUtils.validateUpsertConfig(tableConfig, schema);
+    } catch (Exception e) {
+      Assert.assertEquals(e.getMessage(), "INCREMENT merger should be numeric data types.");

Review comment:
       It won't fail, but still we should not try to increment the timestamp because there is no guarantee that the base value is already ingested (or very hard to manage at least)

##########
File path: pinot-core/src/test/java/org/apache/pinot/core/util/TableConfigUtilsTest.java
##########
@@ -1098,4 +1105,46 @@ public void testValidateUpsertConfig() {
       Assert.assertEquals(e.getMessage(), "The upsert table cannot have star-tree index.");
     }
   }
+
+  @Test
+  public void testValidatePartialUpsertConfig() {
+    Schema schema =
+        new Schema.SchemaBuilder().setSchemaName(TABLE_NAME).addSingleValueDimension("myCol1", FieldSpec.DataType.LONG)
+            .addSingleValueDimension("myCol2", FieldSpec.DataType.STRING)
+            .setPrimaryKeyColumns(Lists.newArrayList("myCol1")).build();
+
+    Map<String, String> streamConfigs = new HashMap<>();
+    streamConfigs.put("stream.kafka.consumer.type", "highLevel");
+    streamConfigs.put("streamType", "kafka");
+    streamConfigs.put("stream.kafka.topic.name", "test");
+    streamConfigs
+        .put("stream.kafka.decoder.class.name", "org.apache.pinot.plugin.stream.kafka.KafkaJSONMessageDecoder");
+    streamConfigs.put("stream.kafka.consumer.type", "simple");
+
+    TableConfig tableConfig = new TableConfigBuilder(TableType.REALTIME).setTableName(TABLE_NAME)
+        .setUpsertConfig(new UpsertConfig(UpsertConfig.Mode.PARTIAL)).setNullHandlingEnabled(false)
+        .setRoutingConfig(new RoutingConfig(null, null, RoutingConfig.STRICT_REPLICA_GROUP_INSTANCE_SELECTOR_TYPE))
+        .setStreamConfigs(streamConfigs).build();
+    try {
+      TableConfigUtils.validateUpsertConfig(tableConfig, schema);
+    } catch (Exception e) {
+      Assert.assertEquals(e.getMessage(), "NullValueHandling is required to be enabled for partial upsert tables.");
+    }
+
+    Map<String, UpsertConfig.Strategy> partialUpsertStratgies = new HashMap<>();
+    partialUpsertStratgies.put("myCol1", UpsertConfig.Strategy.INCREMENT);
+    try {
+      TableConfigUtils.validateUpsertConfig(tableConfig, schema);
+    } catch (Exception e) {
+      Assert.assertEquals(e.getMessage(), "INCREMENT merger cannot be applied to PK.");
+    }
+
+    partialUpsertStratgies = new HashMap<>();
+    partialUpsertStratgies.put("myCol2", UpsertConfig.Strategy.INCREMENT);
+    try {
+      TableConfigUtils.validateUpsertConfig(tableConfig, schema);
+    } catch (Exception e) {
+      Assert.assertEquals(e.getMessage(), "INCREMENT merger should be numeric data types.");

Review comment:
       Yes. IMO wait time should be long, instead of timestamp (milliseconds since epoch)




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

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



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


[GitHub] [incubator-pinot] chenboat commented on a change in pull request #6899: Add partial upsert config and mergers (WIP)

Posted by GitBox <gi...@apache.org>.
chenboat commented on a change in pull request #6899:
URL: https://github.com/apache/incubator-pinot/pull/6899#discussion_r634827105



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/upsert/PartialUpsertMerger.java
##########
@@ -0,0 +1,34 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.core.upsert;
+
+import org.apache.pinot.spi.data.readers.GenericRow;
+
+
+public interface PartialUpsertMerger {
+  /**
+   * Handle partial upsert merge for given fieldName.

Review comment:
       There is no given fieldName in the interface? Delete the reference to 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.

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] [incubator-pinot] codecov-commenter commented on pull request #6899: Add partial upsert config and mergers (WIP)

Posted by GitBox <gi...@apache.org>.
codecov-commenter commented on pull request #6899:
URL: https://github.com/apache/incubator-pinot/pull/6899#issuecomment-853568285


   # [Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6899?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 [#6899](https://codecov.io/gh/apache/incubator-pinot/pull/6899?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (34b10b9) into [master](https://codecov.io/gh/apache/incubator-pinot/commit/0185482d9da2ac299b4b15bcd2998165ccbbdf71?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (0185482) will **decrease** coverage by `7.86%`.
   > The diff coverage is `32.00%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/incubator-pinot/pull/6899/graphs/tree.svg?width=650&height=150&src=pr&token=4ibza2ugkz&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/incubator-pinot/pull/6899?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #6899      +/-   ##
   ============================================
   - Coverage     73.23%   65.37%   -7.87%     
     Complexity       12       12              
   ============================================
     Files          1439     1455      +16     
     Lines         71333    71935     +602     
     Branches      10334    10433      +99     
   ============================================
   - Hits          52243    47029    -5214     
   - Misses        15578    21503    +5925     
   + Partials       3512     3403     -109     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration | `?` | |
   | unittests | `65.37% <32.00%> (+0.02%)` | :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://codecov.io/gh/apache/incubator-pinot/pull/6899?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...ata/manager/realtime/RealtimeTableDataManager.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9kYXRhL21hbmFnZXIvcmVhbHRpbWUvUmVhbHRpbWVUYWJsZURhdGFNYW5hZ2VyLmphdmE=) | `10.74% <0.00%> (-58.04%)` | :arrow_down: |
   | [...not/segment/local/upsert/PartialUpsertHandler.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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==) | `0.00% <0.00%> (ø)` | |
   | [...inot/segment/local/upsert/merger/IgnoreMerger.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC91cHNlcnQvbWVyZ2VyL0lnbm9yZU1lcmdlci5qYXZh) | `0.00% <0.00%> (ø)` | |
   | [...t/segment/local/upsert/merger/IncrementMerger.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC91cHNlcnQvbWVyZ2VyL0luY3JlbWVudE1lcmdlci5qYXZh) | `0.00% <0.00%> (ø)` | |
   | [...t/segment/local/upsert/merger/OverwriteMerger.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC91cHNlcnQvbWVyZ2VyL092ZXJ3cml0ZU1lcmdlci5qYXZh) | `0.00% <0.00%> (ø)` | |
   | [...he/pinot/segment/local/utils/TableConfigUtils.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC91dGlscy9UYWJsZUNvbmZpZ1V0aWxzLmphdmE=) | `75.20% <11.11%> (-6.67%)` | :arrow_down: |
   | [...t/local/upsert/PartitionUpsertMetadataManager.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC91cHNlcnQvUGFydGl0aW9uVXBzZXJ0TWV0YWRhdGFNYW5hZ2VyLmphdmE=) | `84.41% <38.88%> (-10.83%)` | :arrow_down: |
   | [...ent/local/realtime/impl/RealtimeSegmentConfig.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9yZWFsdGltZS9pbXBsL1JlYWx0aW1lU2VnbWVudENvbmZpZy5qYXZh) | `89.43% <55.55%> (-9.70%)` | :arrow_down: |
   | [...rg/apache/pinot/spi/config/table/UpsertConfig.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc3BpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcGkvY29uZmlnL3RhYmxlL1Vwc2VydENvbmZpZy5qYXZh) | `78.26% <73.33%> (-9.24%)` | :arrow_down: |
   | [...local/indexsegment/mutable/MutableSegmentImpl.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9pbmRleHNlZ21lbnQvbXV0YWJsZS9NdXRhYmxlU2VnbWVudEltcGwuamF2YQ==) | `66.21% <88.88%> (-12.65%)` | :arrow_down: |
   | ... and [384 more](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6899?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/incubator-pinot/pull/6899?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 [0185482...34b10b9](https://codecov.io/gh/apache/incubator-pinot/pull/6899?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.

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] [incubator-pinot] codecov-commenter edited a comment on pull request #6899: Add partial upsert config and mergers

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #6899:
URL: https://github.com/apache/incubator-pinot/pull/6899#issuecomment-853568285


   # [Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6899?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 [#6899](https://codecov.io/gh/apache/incubator-pinot/pull/6899?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (32a5f10) into [master](https://codecov.io/gh/apache/incubator-pinot/commit/a1c9b631381a25ddd6d3164d6a9ce337c3939b9f?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (a1c9b63) will **decrease** coverage by `7.94%`.
   > The diff coverage is `32.33%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/incubator-pinot/pull/6899/graphs/tree.svg?width=650&height=150&src=pr&token=4ibza2ugkz&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/incubator-pinot/pull/6899?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #6899      +/-   ##
   ============================================
   - Coverage     73.38%   65.44%   -7.95%     
     Complexity       12       12              
   ============================================
     Files          1453     1457       +4     
     Lines         72032    72139     +107     
     Branches      10427    10449      +22     
   ============================================
   - Hits          52863    47209    -5654     
   - Misses        15643    21513    +5870     
   + Partials       3526     3417     -109     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration | `?` | |
   | unittests | `65.44% <32.33%> (+0.03%)` | :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://codecov.io/gh/apache/incubator-pinot/pull/6899?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...ata/manager/realtime/RealtimeTableDataManager.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9kYXRhL21hbmFnZXIvcmVhbHRpbWUvUmVhbHRpbWVUYWJsZURhdGFNYW5hZ2VyLmphdmE=) | `10.56% <0.00%> (-58.21%)` | :arrow_down: |
   | [...not/segment/local/upsert/PartialUpsertHandler.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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==) | `0.00% <0.00%> (ø)` | |
   | [...inot/segment/local/upsert/merger/IgnoreMerger.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC91cHNlcnQvbWVyZ2VyL0lnbm9yZU1lcmdlci5qYXZh) | `0.00% <0.00%> (ø)` | |
   | [...t/segment/local/upsert/merger/IncrementMerger.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC91cHNlcnQvbWVyZ2VyL0luY3JlbWVudE1lcmdlci5qYXZh) | `0.00% <0.00%> (ø)` | |
   | [...t/segment/local/upsert/merger/OverwriteMerger.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC91cHNlcnQvbWVyZ2VyL092ZXJ3cml0ZU1lcmdlci5qYXZh) | `0.00% <0.00%> (ø)` | |
   | [...he/pinot/segment/local/utils/TableConfigUtils.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC91dGlscy9UYWJsZUNvbmZpZ1V0aWxzLmphdmE=) | `75.20% <11.11%> (-6.67%)` | :arrow_down: |
   | [...t/local/upsert/PartitionUpsertMetadataManager.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC91cHNlcnQvUGFydGl0aW9uVXBzZXJ0TWV0YWRhdGFNYW5hZ2VyLmphdmE=) | `85.18% <52.17%> (-10.06%)` | :arrow_down: |
   | [...rg/apache/pinot/spi/config/table/UpsertConfig.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc3BpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcGkvY29uZmlnL3RhYmxlL1Vwc2VydENvbmZpZy5qYXZh) | `78.26% <73.33%> (-9.24%)` | :arrow_down: |
   | [...local/indexsegment/mutable/MutableSegmentImpl.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC9pbmRleHNlZ21lbnQvbXV0YWJsZS9NdXRhYmxlU2VnbWVudEltcGwuamF2YQ==) | `67.05% <81.25%> (-11.81%)` | :arrow_down: |
   | [...gment/local/upsert/TableUpsertMetadataManager.java](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC91cHNlcnQvVGFibGVVcHNlcnRNZXRhZGF0YU1hbmFnZXIuamF2YQ==) | `100.00% <100.00%> (ø)` | |
   | ... and [356 more](https://codecov.io/gh/apache/incubator-pinot/pull/6899/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) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/6899?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/incubator-pinot/pull/6899?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 [a1c9b63...32a5f10](https://codecov.io/gh/apache/incubator-pinot/pull/6899?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.

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] [incubator-pinot] Jackie-Jiang commented on a change in pull request #6899: Add partial upsert config and mergers

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on a change in pull request #6899:
URL: https://github.com/apache/incubator-pinot/pull/6899#discussion_r650224746



##########
File path: pinot-core/src/test/java/org/apache/pinot/core/util/TableConfigUtilsTest.java
##########
@@ -984,17 +989,19 @@ public void testValidateIndexingConfig() {
       // Expected
     }
 
-    tableConfig = new TableConfigBuilder(TableType.OFFLINE).setTableName(TABLE_NAME).setVarLengthDictionaryColumns(Arrays.asList("intCol")).
-        build();
+    tableConfig = new TableConfigBuilder(TableType.OFFLINE).setTableName(TABLE_NAME)
+        .setVarLengthDictionaryColumns(Arrays.asList("intCol")).

Review comment:
       +1, seems not correctly formatted

##########
File path: pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java
##########
@@ -178,20 +175,20 @@ public MutableSegmentImpl(RealtimeSegmentConfig config, @Nullable ServerMetrics
             realtimeSegmentZKMetadata.getEndTime(), realtimeSegmentZKMetadata.getTimeUnit(),
             realtimeSegmentZKMetadata.getTotalDocs(), realtimeSegmentZKMetadata.getCrc(), _schema) {
           @Override
-      public int getTotalDocs() {
-        return _numDocsIndexed;
-      }
+          public int getTotalDocs() {
+            return _numDocsIndexed;
+          }
 
-      @Override
-      public long getLastIndexedTimestamp() {
-        return _lastIndexedTimeMs;
-      }
+          @Override
+          public long getLastIndexedTimestamp() {
+            return _lastIndexedTimeMs;
+          }
 
-      @Override
-      public long getLatestIngestionTimestamp() {
-        return _latestIngestionTimeMs;
-      }
-    };
+          @Override
+          public long getLatestIngestionTimestamp() {
+            return _latestIngestionTimeMs;

Review comment:
       The old indentation seems wrong though

##########
File path: pinot-segment-local/src/main/java/org/apache/pinot/segment/local/upsert/PartialUpsertHandler.java
##########
@@ -0,0 +1,138 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.segment.local.upsert;
+
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.helix.HelixDataAccessor;
+import org.apache.helix.HelixManager;
+import org.apache.helix.PropertyKey;
+import org.apache.helix.model.CurrentState;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.LiveInstance;
+import org.apache.pinot.segment.local.upsert.merger.PartialUpsertMerger;
+import org.apache.pinot.segment.local.upsert.merger.PartialUpsertMergerFactory;
+import org.apache.pinot.spi.config.table.UpsertConfig;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.utils.CommonConstants.Helix.StateModel.SegmentStateModel;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class PartialUpsertHandler {
+  private static final Logger LOGGER = LoggerFactory.getLogger(PartialUpsertHandler.class);
+
+  private final Map<String, PartialUpsertMerger> _mergers = new HashMap<>();
+
+  private final HelixManager _helixManager;
+  private final String _tableNameWithType;
+  private boolean _allSegmentsLoaded;

Review comment:
       We need it only at server startup, where new consuming segment should wait for all other segments for the table loaded before starting consuming

##########
File path: pinot-core/src/test/java/org/apache/pinot/core/util/TableConfigUtilsTest.java
##########
@@ -984,17 +989,19 @@ public void testValidateIndexingConfig() {
       // Expected
     }
 
-    tableConfig = new TableConfigBuilder(TableType.OFFLINE).setTableName(TABLE_NAME).setVarLengthDictionaryColumns(Arrays.asList("intCol")).
-        build();
+    tableConfig = new TableConfigBuilder(TableType.OFFLINE).setTableName(TABLE_NAME)
+        .setVarLengthDictionaryColumns(Arrays.asList("intCol")).

Review comment:
       @deemoliu https://docs.pinot.apache.org/developers/developers-and-contributors/code-setup

##########
File path: pinot-segment-local/src/main/java/org/apache/pinot/segment/local/upsert/PartialUpsertHandler.java
##########
@@ -0,0 +1,138 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.segment.local.upsert;
+
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.helix.HelixDataAccessor;
+import org.apache.helix.HelixManager;
+import org.apache.helix.PropertyKey;
+import org.apache.helix.model.CurrentState;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.LiveInstance;
+import org.apache.pinot.segment.local.upsert.merger.PartialUpsertMerger;
+import org.apache.pinot.segment.local.upsert.merger.PartialUpsertMergerFactory;
+import org.apache.pinot.spi.config.table.UpsertConfig;
+import org.apache.pinot.spi.data.readers.GenericRow;
+import org.apache.pinot.spi.utils.CommonConstants.Helix.StateModel.SegmentStateModel;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class PartialUpsertHandler {
+  private static final Logger LOGGER = LoggerFactory.getLogger(PartialUpsertHandler.class);
+
+  private final Map<String, PartialUpsertMerger> _mergers = new HashMap<>();
+
+  private final HelixManager _helixManager;
+  private final String _tableNameWithType;
+  private boolean _allSegmentsLoaded;

Review comment:
       That won't work because basically that is changing the history. The change must come from the stream events

##########
File path: pinot-segment-local/src/main/java/org/apache/pinot/segment/local/upsert/PartitionUpsertMetadataManager.java
##########
@@ -60,128 +64,175 @@
   private final String _tableNameWithType;
   private final int _partitionId;
   private final ServerMetrics _serverMetrics;
+  private final PartialUpsertHandler _partialUpsertHandler;
 
-  public PartitionUpsertMetadataManager(String tableNameWithType, int partitionId, ServerMetrics serverMetrics) {
+  // TODO(upset): consider an off-heap KV store to persist this index to improve the recovery speed.
+  @VisibleForTesting
+  final ConcurrentHashMap<PrimaryKey, RecordLocation> _primaryKeyToRecordLocationMap = new ConcurrentHashMap<>();
+
+  // Reused for reading previous record during partial upsert
+  private final GenericRow _reuse = new GenericRow();
+  // Stores the result of updateRecord()
+  private GenericRow _result;
+
+  public PartitionUpsertMetadataManager(String tableNameWithType, int partitionId, ServerMetrics serverMetrics,
+      @Nullable PartialUpsertHandler partialUpsertHandler) {
     _tableNameWithType = tableNameWithType;
     _partitionId = partitionId;
     _serverMetrics = serverMetrics;
+    _partialUpsertHandler = partialUpsertHandler;
   }
 
-  public ConcurrentHashMap<PrimaryKey, RecordLocation> getPrimaryKeyToRecordLocationMap() {
-    return _primaryKeyToRecordLocationMap;
+  /**
+   * Initializes the upsert metadata for the given immutable segment.
+   */
+  public void addSegment(ImmutableSegmentImpl immutableSegment, Iterator<RecordInfo> recordInfoIterator) {
+    addSegment(immutableSegment, recordInfoIterator, new ThreadSafeMutableRoaringBitmap());
   }
 
-  // TODO(upset): consider an off-heap KV store to persist this index to improve the recovery speed.
   @VisibleForTesting
-  final ConcurrentHashMap<PrimaryKey, RecordLocation> _primaryKeyToRecordLocationMap = new ConcurrentHashMap<>();
-
-  /**
-   * Initializes the upsert metadata for the given immutable segment, returns the valid doc ids for the segment.
-   */
-  public ThreadSafeMutableRoaringBitmap addSegment(String segmentName, Iterator<RecordInfo> recordInfoIterator) {
+  void addSegment(ImmutableSegmentImpl immutableSegment, Iterator<RecordInfo> recordInfoIterator,
+      ThreadSafeMutableRoaringBitmap validDocIds) {
+    String segmentName = immutableSegment.getSegmentName();
     LOGGER.info("Adding upsert metadata for segment: {}", segmentName);
+    immutableSegment.enableUpsert(this, validDocIds);
 
-    ThreadSafeMutableRoaringBitmap validDocIds = new ThreadSafeMutableRoaringBitmap();
     while (recordInfoIterator.hasNext()) {
       RecordInfo recordInfo = recordInfoIterator.next();
       _primaryKeyToRecordLocationMap.compute(recordInfo._primaryKey, (primaryKey, currentRecordLocation) -> {
         if (currentRecordLocation != null) {
           // Existing primary key
 
-          if (segmentName.equals(currentRecordLocation.getSegmentName())) {
-            // The current record location has the same segment name
-
-            // Update the record location when the new timestamp is greater than or equal to the current timestamp.
-            // There are 2 scenarios:
-            //   1. The current record location is pointing to the same segment (the segment being added). In this case,
-            //      we want to update the record location when there is a tie to keep the newer record. Note that the
-            //      record info iterator will return records with incremental doc ids.
-            //   2. The current record location is pointing to the old segment being replaced. This could happen when
-            //      committing a consuming segment, or reloading a completed segment. In this case, we want to update
-            //      the record location when there is a tie because the record locations should point to the new added
-            //      segment instead of the old segment being replaced. Also, do not update the valid doc ids for the old
-            //      segment because it has not been replaced yet.
+          // The current record is in the same segment
+          // Update the record location when there is a tie to keep the newer record. Note that the record info iterator
+          // will return records with incremental doc ids.
+          IndexSegment currentSegment = currentRecordLocation.getSegment();
+          if (immutableSegment == currentSegment) {
             if (recordInfo._timestamp >= currentRecordLocation.getTimestamp()) {
-              // Only update the valid doc ids for the new segment
-              if (validDocIds == currentRecordLocation.getValidDocIds()) {
-                validDocIds.remove(currentRecordLocation.getDocId());
-              }
+              validDocIds.remove(currentRecordLocation.getDocId());
               validDocIds.add(recordInfo._docId);
-              return new RecordLocation(segmentName, recordInfo._docId, recordInfo._timestamp, validDocIds);
+              return new RecordLocation(immutableSegment, recordInfo._docId, recordInfo._timestamp);
             } else {
               return currentRecordLocation;
             }
-          } else {
-            // The current record location is pointing to a different segment
-
-            // Update the record location when getting a newer timestamp, or the timestamp is the same as the current
-            // timestamp, but the segment has a larger sequence number (the segment is newer than the current segment).
-            if (recordInfo._timestamp > currentRecordLocation.getTimestamp() || (
-                recordInfo._timestamp == currentRecordLocation.getTimestamp()
-                    && LLCSegmentName.isLowLevelConsumerSegmentName(segmentName)
-                    && LLCSegmentName.isLowLevelConsumerSegmentName(currentRecordLocation.getSegmentName())
-                    && LLCSegmentName.getSequenceNumber(segmentName) > LLCSegmentName
-                    .getSequenceNumber(currentRecordLocation.getSegmentName()))) {
-              currentRecordLocation.getValidDocIds().remove(currentRecordLocation.getDocId());
+          }
+
+          // The current record is in an old segment being replaced
+          // This could happen when committing a consuming segment, or reloading a completed segment. In this case, we
+          // want to update the record location when there is a tie because the record locations should point to the new
+          // added segment instead of the old segment being replaced. Also, do not update the valid doc ids for the old
+          // segment because it has not been replaced yet.
+          String currentSegmentName = currentSegment.getSegmentName();
+          if (segmentName.equals(currentSegmentName)) {
+            if (recordInfo._timestamp >= currentRecordLocation.getTimestamp()) {
               validDocIds.add(recordInfo._docId);
-              return new RecordLocation(segmentName, recordInfo._docId, recordInfo._timestamp, validDocIds);
+              return new RecordLocation(immutableSegment, recordInfo._docId, recordInfo._timestamp);
             } else {
               return currentRecordLocation;
             }
           }
+
+          // The current record is in a different segment
+          // Update the record location when getting a newer timestamp, or the timestamp is the same as the current
+          // timestamp, but the segment has a larger sequence number (the segment is newer than the current segment).
+          if (recordInfo._timestamp > currentRecordLocation.getTimestamp() || (
+              recordInfo._timestamp == currentRecordLocation.getTimestamp() && LLCSegmentName
+                  .isLowLevelConsumerSegmentName(segmentName) && LLCSegmentName
+                  .isLowLevelConsumerSegmentName(currentSegmentName)
+                  && LLCSegmentName.getSequenceNumber(segmentName) > LLCSegmentName
+                  .getSequenceNumber(currentSegmentName))) {
+            assert currentSegment.getValidDocIds() != null;
+            currentSegment.getValidDocIds().remove(currentRecordLocation.getDocId());
+            validDocIds.add(recordInfo._docId);
+            return new RecordLocation(immutableSegment, recordInfo._docId, recordInfo._timestamp);
+          } else {
+            return currentRecordLocation;
+          }
         } else {
           // New primary key
           validDocIds.add(recordInfo._docId);
-          return new RecordLocation(segmentName, recordInfo._docId, recordInfo._timestamp, validDocIds);
+          return new RecordLocation(immutableSegment, recordInfo._docId, recordInfo._timestamp);
         }
       });
     }
     // Update metrics
     _serverMetrics.setValueOfPartitionGauge(_tableNameWithType, _partitionId, ServerGauge.UPSERT_PRIMARY_KEYS_COUNT,
         _primaryKeyToRecordLocationMap.size());
-    return validDocIds;
   }
 
   /**
-   * Updates the upsert metadata for a new consumed record in the given consuming segment.
+   * Updates the upsert metadata for a new consumed record in the given consuming segment. Returns the merged record if
+   * partial-upsert is enabled.
    */
-  public void updateRecord(String segmentName, RecordInfo recordInfo, ThreadSafeMutableRoaringBitmap validDocIds) {
+  public GenericRow updateRecord(IndexSegment segment, RecordInfo recordInfo, GenericRow record) {
+    // For partial-upsert, need to ensure all previous records are loaded before inserting new records.
+    if (_partialUpsertHandler != null) {
+      while (!_partialUpsertHandler.isAllSegmentsLoaded()) {
+        LOGGER
+            .info("Sleeping 1 second waiting for all segments loaded for partial-upsert table: {}", _tableNameWithType);

Review comment:
       This is blocking the ingestion, so it won't seal while waiting

##########
File path: pinot-segment-local/src/main/java/org/apache/pinot/segment/local/upsert/merger/IncrementMerger.java
##########
@@ -0,0 +1,44 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.segment.local.upsert.merger;
+
+public class IncrementMerger implements PartialUpsertMerger {
+  IncrementMerger() {

Review comment:
       The factory creates one instance of each merger, so it is package private

##########
File path: pinot-segment-local/src/main/java/org/apache/pinot/segment/local/upsert/PartitionUpsertMetadataManager.java
##########
@@ -60,128 +64,175 @@
   private final String _tableNameWithType;
   private final int _partitionId;
   private final ServerMetrics _serverMetrics;
+  private final PartialUpsertHandler _partialUpsertHandler;
 
-  public PartitionUpsertMetadataManager(String tableNameWithType, int partitionId, ServerMetrics serverMetrics) {
+  // TODO(upset): consider an off-heap KV store to persist this index to improve the recovery speed.
+  @VisibleForTesting
+  final ConcurrentHashMap<PrimaryKey, RecordLocation> _primaryKeyToRecordLocationMap = new ConcurrentHashMap<>();
+
+  // Reused for reading previous record during partial upsert
+  private final GenericRow _reuse = new GenericRow();
+  // Stores the result of updateRecord()
+  private GenericRow _result;
+
+  public PartitionUpsertMetadataManager(String tableNameWithType, int partitionId, ServerMetrics serverMetrics,
+      @Nullable PartialUpsertHandler partialUpsertHandler) {
     _tableNameWithType = tableNameWithType;
     _partitionId = partitionId;
     _serverMetrics = serverMetrics;
+    _partialUpsertHandler = partialUpsertHandler;
   }
 
-  public ConcurrentHashMap<PrimaryKey, RecordLocation> getPrimaryKeyToRecordLocationMap() {
-    return _primaryKeyToRecordLocationMap;
+  /**
+   * Initializes the upsert metadata for the given immutable segment.
+   */
+  public void addSegment(ImmutableSegmentImpl immutableSegment, Iterator<RecordInfo> recordInfoIterator) {
+    addSegment(immutableSegment, recordInfoIterator, new ThreadSafeMutableRoaringBitmap());
   }
 
-  // TODO(upset): consider an off-heap KV store to persist this index to improve the recovery speed.
   @VisibleForTesting
-  final ConcurrentHashMap<PrimaryKey, RecordLocation> _primaryKeyToRecordLocationMap = new ConcurrentHashMap<>();
-
-  /**
-   * Initializes the upsert metadata for the given immutable segment, returns the valid doc ids for the segment.
-   */
-  public ThreadSafeMutableRoaringBitmap addSegment(String segmentName, Iterator<RecordInfo> recordInfoIterator) {
+  void addSegment(ImmutableSegmentImpl immutableSegment, Iterator<RecordInfo> recordInfoIterator,
+      ThreadSafeMutableRoaringBitmap validDocIds) {
+    String segmentName = immutableSegment.getSegmentName();
     LOGGER.info("Adding upsert metadata for segment: {}", segmentName);
+    immutableSegment.enableUpsert(this, validDocIds);
 
-    ThreadSafeMutableRoaringBitmap validDocIds = new ThreadSafeMutableRoaringBitmap();
     while (recordInfoIterator.hasNext()) {
       RecordInfo recordInfo = recordInfoIterator.next();
       _primaryKeyToRecordLocationMap.compute(recordInfo._primaryKey, (primaryKey, currentRecordLocation) -> {
         if (currentRecordLocation != null) {
           // Existing primary key
 
-          if (segmentName.equals(currentRecordLocation.getSegmentName())) {
-            // The current record location has the same segment name
-
-            // Update the record location when the new timestamp is greater than or equal to the current timestamp.
-            // There are 2 scenarios:
-            //   1. The current record location is pointing to the same segment (the segment being added). In this case,
-            //      we want to update the record location when there is a tie to keep the newer record. Note that the
-            //      record info iterator will return records with incremental doc ids.
-            //   2. The current record location is pointing to the old segment being replaced. This could happen when
-            //      committing a consuming segment, or reloading a completed segment. In this case, we want to update
-            //      the record location when there is a tie because the record locations should point to the new added
-            //      segment instead of the old segment being replaced. Also, do not update the valid doc ids for the old
-            //      segment because it has not been replaced yet.
+          // The current record is in the same segment
+          // Update the record location when there is a tie to keep the newer record. Note that the record info iterator
+          // will return records with incremental doc ids.
+          IndexSegment currentSegment = currentRecordLocation.getSegment();
+          if (immutableSegment == currentSegment) {
             if (recordInfo._timestamp >= currentRecordLocation.getTimestamp()) {
-              // Only update the valid doc ids for the new segment
-              if (validDocIds == currentRecordLocation.getValidDocIds()) {
-                validDocIds.remove(currentRecordLocation.getDocId());
-              }
+              validDocIds.remove(currentRecordLocation.getDocId());
               validDocIds.add(recordInfo._docId);
-              return new RecordLocation(segmentName, recordInfo._docId, recordInfo._timestamp, validDocIds);
+              return new RecordLocation(immutableSegment, recordInfo._docId, recordInfo._timestamp);
             } else {
               return currentRecordLocation;
             }
-          } else {
-            // The current record location is pointing to a different segment
-
-            // Update the record location when getting a newer timestamp, or the timestamp is the same as the current
-            // timestamp, but the segment has a larger sequence number (the segment is newer than the current segment).
-            if (recordInfo._timestamp > currentRecordLocation.getTimestamp() || (
-                recordInfo._timestamp == currentRecordLocation.getTimestamp()
-                    && LLCSegmentName.isLowLevelConsumerSegmentName(segmentName)
-                    && LLCSegmentName.isLowLevelConsumerSegmentName(currentRecordLocation.getSegmentName())
-                    && LLCSegmentName.getSequenceNumber(segmentName) > LLCSegmentName
-                    .getSequenceNumber(currentRecordLocation.getSegmentName()))) {
-              currentRecordLocation.getValidDocIds().remove(currentRecordLocation.getDocId());
+          }
+
+          // The current record is in an old segment being replaced
+          // This could happen when committing a consuming segment, or reloading a completed segment. In this case, we
+          // want to update the record location when there is a tie because the record locations should point to the new
+          // added segment instead of the old segment being replaced. Also, do not update the valid doc ids for the old
+          // segment because it has not been replaced yet.
+          String currentSegmentName = currentSegment.getSegmentName();
+          if (segmentName.equals(currentSegmentName)) {
+            if (recordInfo._timestamp >= currentRecordLocation.getTimestamp()) {
               validDocIds.add(recordInfo._docId);
-              return new RecordLocation(segmentName, recordInfo._docId, recordInfo._timestamp, validDocIds);
+              return new RecordLocation(immutableSegment, recordInfo._docId, recordInfo._timestamp);
             } else {
               return currentRecordLocation;
             }
           }
+
+          // The current record is in a different segment
+          // Update the record location when getting a newer timestamp, or the timestamp is the same as the current
+          // timestamp, but the segment has a larger sequence number (the segment is newer than the current segment).
+          if (recordInfo._timestamp > currentRecordLocation.getTimestamp() || (
+              recordInfo._timestamp == currentRecordLocation.getTimestamp() && LLCSegmentName
+                  .isLowLevelConsumerSegmentName(segmentName) && LLCSegmentName
+                  .isLowLevelConsumerSegmentName(currentSegmentName)
+                  && LLCSegmentName.getSequenceNumber(segmentName) > LLCSegmentName
+                  .getSequenceNumber(currentSegmentName))) {
+            assert currentSegment.getValidDocIds() != null;
+            currentSegment.getValidDocIds().remove(currentRecordLocation.getDocId());
+            validDocIds.add(recordInfo._docId);
+            return new RecordLocation(immutableSegment, recordInfo._docId, recordInfo._timestamp);
+          } else {
+            return currentRecordLocation;
+          }
         } else {
           // New primary key
           validDocIds.add(recordInfo._docId);
-          return new RecordLocation(segmentName, recordInfo._docId, recordInfo._timestamp, validDocIds);
+          return new RecordLocation(immutableSegment, recordInfo._docId, recordInfo._timestamp);
         }
       });
     }
     // Update metrics
     _serverMetrics.setValueOfPartitionGauge(_tableNameWithType, _partitionId, ServerGauge.UPSERT_PRIMARY_KEYS_COUNT,
         _primaryKeyToRecordLocationMap.size());
-    return validDocIds;
   }
 
   /**
-   * Updates the upsert metadata for a new consumed record in the given consuming segment.
+   * Updates the upsert metadata for a new consumed record in the given consuming segment. Returns the merged record if
+   * partial-upsert is enabled.
    */
-  public void updateRecord(String segmentName, RecordInfo recordInfo, ThreadSafeMutableRoaringBitmap validDocIds) {
+  public GenericRow updateRecord(IndexSegment segment, RecordInfo recordInfo, GenericRow record) {
+    // For partial-upsert, need to ensure all previous records are loaded before inserting new records.
+    if (_partialUpsertHandler != null) {
+      while (!_partialUpsertHandler.isAllSegmentsLoaded()) {
+        LOGGER
+            .info("Sleeping 1 second waiting for all segments loaded for partial-upsert table: {}", _tableNameWithType);
+        try {
+          //noinspection BusyWait
+          Thread.sleep(1000L);
+        } catch (InterruptedException e) {
+          throw new RuntimeException(e);
+        }
+      }
+    }
+
+    _result = record;
     _primaryKeyToRecordLocationMap.compute(recordInfo._primaryKey, (primaryKey, currentRecordLocation) -> {
       if (currentRecordLocation != null) {
         // Existing primary key
 
         // Update the record location when the new timestamp is greater than or equal to the current timestamp. Update
         // the record location when there is a tie to keep the newer record.
         if (recordInfo._timestamp >= currentRecordLocation.getTimestamp()) {
-          currentRecordLocation.getValidDocIds().remove(currentRecordLocation.getDocId());
-          validDocIds.add(recordInfo._docId);
-          return new RecordLocation(segmentName, recordInfo._docId, recordInfo._timestamp, validDocIds);
+          IndexSegment currentSegment = currentRecordLocation.getSegment();
+          if (_partialUpsertHandler != null) {
+            // Partial upsert
+            GenericRow previousRecord = currentSegment.getRecord(currentRecordLocation.getDocId(), _reuse);

Review comment:
       No, if record location is not null, then we should be able to read the previous record.
   `IndexSegment.getRecord()` will never return null

##########
File path: pinot-segment-local/src/main/java/org/apache/pinot/segment/local/upsert/PartitionUpsertMetadataManager.java
##########
@@ -60,128 +64,175 @@
   private final String _tableNameWithType;
   private final int _partitionId;
   private final ServerMetrics _serverMetrics;
+  private final PartialUpsertHandler _partialUpsertHandler;
 
-  public PartitionUpsertMetadataManager(String tableNameWithType, int partitionId, ServerMetrics serverMetrics) {
+  // TODO(upset): consider an off-heap KV store to persist this index to improve the recovery speed.
+  @VisibleForTesting
+  final ConcurrentHashMap<PrimaryKey, RecordLocation> _primaryKeyToRecordLocationMap = new ConcurrentHashMap<>();
+
+  // Reused for reading previous record during partial upsert
+  private final GenericRow _reuse = new GenericRow();
+  // Stores the result of updateRecord()
+  private GenericRow _result;
+
+  public PartitionUpsertMetadataManager(String tableNameWithType, int partitionId, ServerMetrics serverMetrics,
+      @Nullable PartialUpsertHandler partialUpsertHandler) {
     _tableNameWithType = tableNameWithType;
     _partitionId = partitionId;
     _serverMetrics = serverMetrics;
+    _partialUpsertHandler = partialUpsertHandler;
   }
 
-  public ConcurrentHashMap<PrimaryKey, RecordLocation> getPrimaryKeyToRecordLocationMap() {
-    return _primaryKeyToRecordLocationMap;
+  /**
+   * Initializes the upsert metadata for the given immutable segment.
+   */
+  public void addSegment(ImmutableSegmentImpl immutableSegment, Iterator<RecordInfo> recordInfoIterator) {
+    addSegment(immutableSegment, recordInfoIterator, new ThreadSafeMutableRoaringBitmap());
   }
 
-  // TODO(upset): consider an off-heap KV store to persist this index to improve the recovery speed.
   @VisibleForTesting
-  final ConcurrentHashMap<PrimaryKey, RecordLocation> _primaryKeyToRecordLocationMap = new ConcurrentHashMap<>();
-
-  /**
-   * Initializes the upsert metadata for the given immutable segment, returns the valid doc ids for the segment.
-   */
-  public ThreadSafeMutableRoaringBitmap addSegment(String segmentName, Iterator<RecordInfo> recordInfoIterator) {
+  void addSegment(ImmutableSegmentImpl immutableSegment, Iterator<RecordInfo> recordInfoIterator,
+      ThreadSafeMutableRoaringBitmap validDocIds) {
+    String segmentName = immutableSegment.getSegmentName();
     LOGGER.info("Adding upsert metadata for segment: {}", segmentName);
+    immutableSegment.enableUpsert(this, validDocIds);
 
-    ThreadSafeMutableRoaringBitmap validDocIds = new ThreadSafeMutableRoaringBitmap();
     while (recordInfoIterator.hasNext()) {
       RecordInfo recordInfo = recordInfoIterator.next();
       _primaryKeyToRecordLocationMap.compute(recordInfo._primaryKey, (primaryKey, currentRecordLocation) -> {
         if (currentRecordLocation != null) {
           // Existing primary key
 
-          if (segmentName.equals(currentRecordLocation.getSegmentName())) {
-            // The current record location has the same segment name
-
-            // Update the record location when the new timestamp is greater than or equal to the current timestamp.
-            // There are 2 scenarios:
-            //   1. The current record location is pointing to the same segment (the segment being added). In this case,
-            //      we want to update the record location when there is a tie to keep the newer record. Note that the
-            //      record info iterator will return records with incremental doc ids.
-            //   2. The current record location is pointing to the old segment being replaced. This could happen when
-            //      committing a consuming segment, or reloading a completed segment. In this case, we want to update
-            //      the record location when there is a tie because the record locations should point to the new added
-            //      segment instead of the old segment being replaced. Also, do not update the valid doc ids for the old
-            //      segment because it has not been replaced yet.
+          // The current record is in the same segment
+          // Update the record location when there is a tie to keep the newer record. Note that the record info iterator
+          // will return records with incremental doc ids.
+          IndexSegment currentSegment = currentRecordLocation.getSegment();
+          if (immutableSegment == currentSegment) {
             if (recordInfo._timestamp >= currentRecordLocation.getTimestamp()) {
-              // Only update the valid doc ids for the new segment
-              if (validDocIds == currentRecordLocation.getValidDocIds()) {
-                validDocIds.remove(currentRecordLocation.getDocId());
-              }
+              validDocIds.remove(currentRecordLocation.getDocId());
               validDocIds.add(recordInfo._docId);
-              return new RecordLocation(segmentName, recordInfo._docId, recordInfo._timestamp, validDocIds);
+              return new RecordLocation(immutableSegment, recordInfo._docId, recordInfo._timestamp);
             } else {
               return currentRecordLocation;
             }
-          } else {
-            // The current record location is pointing to a different segment
-
-            // Update the record location when getting a newer timestamp, or the timestamp is the same as the current
-            // timestamp, but the segment has a larger sequence number (the segment is newer than the current segment).
-            if (recordInfo._timestamp > currentRecordLocation.getTimestamp() || (
-                recordInfo._timestamp == currentRecordLocation.getTimestamp()
-                    && LLCSegmentName.isLowLevelConsumerSegmentName(segmentName)
-                    && LLCSegmentName.isLowLevelConsumerSegmentName(currentRecordLocation.getSegmentName())
-                    && LLCSegmentName.getSequenceNumber(segmentName) > LLCSegmentName
-                    .getSequenceNumber(currentRecordLocation.getSegmentName()))) {
-              currentRecordLocation.getValidDocIds().remove(currentRecordLocation.getDocId());
+          }
+
+          // The current record is in an old segment being replaced
+          // This could happen when committing a consuming segment, or reloading a completed segment. In this case, we
+          // want to update the record location when there is a tie because the record locations should point to the new
+          // added segment instead of the old segment being replaced. Also, do not update the valid doc ids for the old
+          // segment because it has not been replaced yet.
+          String currentSegmentName = currentSegment.getSegmentName();
+          if (segmentName.equals(currentSegmentName)) {
+            if (recordInfo._timestamp >= currentRecordLocation.getTimestamp()) {
               validDocIds.add(recordInfo._docId);
-              return new RecordLocation(segmentName, recordInfo._docId, recordInfo._timestamp, validDocIds);
+              return new RecordLocation(immutableSegment, recordInfo._docId, recordInfo._timestamp);
             } else {
               return currentRecordLocation;
             }
           }
+
+          // The current record is in a different segment
+          // Update the record location when getting a newer timestamp, or the timestamp is the same as the current
+          // timestamp, but the segment has a larger sequence number (the segment is newer than the current segment).

Review comment:
       Partial upsert requires storing the actual segment instead of the segment name because it requires reading the previous record from the segment.
   The handling logic is the same as full upsert, where the immutable segment (completed segment) is loaded via the `addSegment()` (per-segment load), and the consuming segment is loaded via the `updateRecord()` (per-record load).




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

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



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